[Patch] Optimize dropping of relation buffers using dlist

Started by k.jamison@fujitsu.comabout 6 years ago271 messages
#1k.jamison@fujitsu.com
k.jamison@fujitsu.com
1 attachment(s)

Hi,

Currently, we need to scan the WHOLE shared buffers when VACUUM
truncated off any empty pages at end of transaction or when relation
is TRUNCATEd.
As for our customer case, we periodically truncate thousands of tables,
and it's possible to TRUNCATE single table per transaction. This can be
problematic later on during recovery which could take longer, especially
when a sudden failover happens after those TRUNCATEs and when we
have to scan a large-sized shared buffer. In the performance test below,
it took almost 12.5 minutes for recovery to complete for 100GB shared
buffers. But we want to keep failover very short (within 10 seconds).

Previously, I made an improvement in speeding the truncates of relation
forks from 3 scans to one scan.[1]/messages/by-id/D09B13F772D2274BB348A310EE3027C64E2067@g01jpexmbkw24 This time, the aim of this patch is
to further speedup the invalidation of pages, by linking the cached pages
of the target relation in a doubly-linked list and just traversing it
instead of scanning the whole shared buffers. In DropRelFileNodeBuffers,
we just get the number of target buffers to invalidate for the relation.
There is a significant win in this patch, because we were able to
complete failover and recover in 3 seconds more or less.

I performed similar tests to what I did in the speedup truncates of
relations forks.[1]/messages/by-id/D09B13F772D2274BB348A310EE3027C64E2067@g01jpexmbkw24[2]/messages/by-id/D09B13F772D2274BB348A310EE3027C6502672@g01jpexmbkw24 However, this time using 100GB shared_buffers.

[Machine spec used in testing]
Intel(R) Xeon(R) CPU E5-2667 v3 @ 3.20GHz
CPU: 16, Number of cores per socket: 8
RHEL6.5, Memory: 256GB++

[Test]
1. (Master) Create table (ex. 10,000 tables). Insert data to tables.
2. (Master) DELETE FROM TABLE (ex. all rows of 10,000 tables)
(Standby) To test with failover, pause the WAL replay on standby server.
(SELECT pg_wal_replay_pause();)
3. (M) psql -c "\timing on" (measures total execution of SQL queries)
4. (M) VACUUM (whole db)
5. (M) Stop primary server. pg_ctl stop -D $PGDATA -w
6. (S) Resume wal replay and promote standby.[2]/messages/by-id/D09B13F772D2274BB348A310EE3027C6502672@g01jpexmbkw24

[Results]

A. HEAD (origin/master branch)
A1. Vacuum execution on Primary server
Time: 730932.408 ms (12:10.932) ~12min 11s
A2. Vacuum + Failover (WAL Recovery on Standby)
waiting for server to promote...........................
.................................... stopped waiting
pg_ctl: server did not promote in time
2019/10/25_12:13:09.692─┐
2019/10/25_12:25:43.576─┘
-->Total: 12min34s

B. PATCH
B1. Vacuum execution on Primary/Master
Time: 6.518333s = 6518.333 ms
B2. Vacuum + Failover (WAL Recovery on Standby)
2019/10/25_14:17:21.822
waiting for server to promote...... done
server promoted
2019/10/25_14:17:24.827
2019/10/25_14:17:24.833
-->Total: 3.011s

[Other Notes]
Maybe one disadvantage is that we can have a variable number of
relations, and allocated the same number of relation structures as
the size of shared buffers. I tried to reduce the use of memory when
doing hash table lookup operation by having a fixed size array (100)
or threshold of target buffers to invalidate.
When doing CachedBufLookup() to scan the count of each buffer in the
dlist, I made sure to reduce the number of scans (2x at most).
First, we scan the dlist of cached buffers of relations.
Then store the target buffers in buf_id_array. Non-target buffers
would be removed from dlist but added to temporary dlist.
After reaching end of main dlist, we append the temporary dlist to
tail of main dlist.
I also performed pgbench buffer test, and this patch did not cause
overhead to normal DB access performance.

Another one that I'd need feedback of is the use of new dlist operations
for this cached buffer list. I did not use in this patch the existing
Postgres dlist architecture (ilist.h) because I want to save memory space
as much as possible especially when NBuffers become large. Both dlist_node
& dlist_head are 16 bytes. OTOH, two int pointers for this patch is 8 bytes.

Hope to hear your feedback and comments.

Thanks in advance,
Kirk Jamison

[1]: /messages/by-id/D09B13F772D2274BB348A310EE3027C64E2067@g01jpexmbkw24
[2]: /messages/by-id/D09B13F772D2274BB348A310EE3027C6502672@g01jpexmbkw24

Attachments:

v1-Optimize-dropping-of-relation-buffers-using-dlist.patchapplication/octet-stream; name=v1-Optimize-dropping-of-relation-buffers-using-dlist.patchDownload
From db038ba47237b47b5eaa6548649907de6e8d8196 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 25 Sep 2019 02:24:13 +0000
Subject: [PATCH] Optimize dropping of relation buffers using dlist

---
 src/backend/storage/buffer/Makefile     |   2 +-
 src/backend/storage/buffer/buf_init.c   |   6 +
 src/backend/storage/buffer/bufmgr.c     |  84 +++++----
 src/backend/storage/buffer/cached_buf.c | 324 ++++++++++++++++++++++++++++++++
 src/include/storage/buf_internals.h     | 151 +++++++++++++++
 5 files changed, 527 insertions(+), 40 deletions(-)
 create mode 100644 src/backend/storage/buffer/cached_buf.c

diff --git a/src/backend/storage/buffer/Makefile b/src/backend/storage/buffer/Makefile
index 2c10fba..fa23c0c 100644
--- a/src/backend/storage/buffer/Makefile
+++ b/src/backend/storage/buffer/Makefile
@@ -12,6 +12,6 @@ subdir = src/backend/storage/buffer
 top_builddir = ../../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = buf_table.o buf_init.o bufmgr.o freelist.o localbuf.o
+OBJS = buf_table.o buf_init.o bufmgr.o freelist.o localbuf.o cached_buf.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/storage/buffer/buf_init.c b/src/backend/storage/buffer/buf_init.c
index ccd2c31..3e4eb40 100644
--- a/src/backend/storage/buffer/buf_init.c
+++ b/src/backend/storage/buffer/buf_init.c
@@ -146,6 +146,9 @@ InitBufferPool(void)
 	/* Init other shared buffer-management stuff */
 	StrategyInitialize(!foundDescs);
 
+	/* Init cached buffer hash table and related shmem data structures */
+	InitCachedBufTable(NBuffers);
+
 	/* Initialize per-backend file flush context */
 	WritebackContextInit(&BackendWritebackContext,
 						 &backend_flush_after);
@@ -189,5 +192,8 @@ BufferShmemSize(void)
 	/* size of checkpoint sort array in bufmgr.c */
 	size = add_size(size, mul_size(NBuffers, sizeof(CkptSortItem)));
 
+	/* size of cached buffer shmem data structures */
+	size = add_size(size, CachedBufShmemSize());
+
 	return size;
 }
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 483f705..036ad6a 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -1268,6 +1268,8 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 			return buf;
 		}
+		/* Insert an entry into the cached block list */
+		CachedBufTableInsert(&newTag.rnode, buf->buf_id);
 
 		/*
 		 * Need to lock the buffer header too in order to change its tag.
@@ -1285,6 +1287,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 			break;
 
 		UnlockBufHdr(buf, buf_state);
+		CachedBufTableDelete(&newTag.rnode, buf->buf_id);
 		BufTableDelete(&newTag, newHash);
 		if (oldPartitionLock != NULL &&
 			oldPartitionLock != newPartitionLock)
@@ -1319,6 +1322,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	if (oldPartitionLock != NULL)
 	{
+		CachedBufTableDelete(&oldTag.rnode, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
 		if (oldPartitionLock != newPartitionLock)
 			LWLockRelease(oldPartitionLock);
@@ -1432,7 +1436,10 @@ retry:
 	 * Remove the buffer from the lookup hashtable, if it was in there.
 	 */
 	if (oldFlags & BM_TAG_VALID)
+	{
+		CachedBufTableDelete(&oldTag.rnode, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
+	}
 
 	/*
 	 * Done with mapping lock.
@@ -2916,19 +2923,15 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
 DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
-	int			i;
-	int			j;
+	int			i, j, nbufs;
+	int			buf_id_array[BUF_ID_ARRAY_SIZE];
+	int			forknum_indexes[BUF_ID_ARRAY_SIZE];
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2942,45 +2945,48 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	do
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		nbufs = CachedBufLookup(rnode.node, forkNum, nforks,
+								  forknum_indexes, firstDelBlock,
+								  buf_id_array, lengthof(buf_id_array));
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+		for (i = 0; i < nbufs; i++)
+		{
+			BufferDesc *bufHdr = GetBufferDescriptor(buf_id_array[i]);
+			uint32	buf_state;
+			int		index = forknum_indexes[i];
 
-		buf_state = LockBufHdr(bufHdr);
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag
+			 * before we attempt to lock the buffer; this saves a lot of
+			 * lock acquisitions in typical cases.  It should be safe
+			 * because the caller must have AccessExclusiveLock on the
+			 * relation, or some other reason to be certain that no one is
+			 * loading new pages of the rel into the buffer pool.
+			 * (Otherwise we might well miss such pages entirely.)
+			 * Therefore, while the tag might be changing while we look at
+			 * it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible,
+			 * and false positives are safe because we'll recheck after
+			 * getting the buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but
+			 * the incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
+
+			buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
-		{
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
-			{
+				bufHdr->tag.forkNum == forkNum[index] &&
+				bufHdr->tag.blockNum >= firstDelBlock[index])
 				InvalidateBuffer(bufHdr); /* releases spinlock */
-				break;
-			}
+			else
+				UnlockBufHdr(bufHdr, buf_state);
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
-	}
+	} while (nbufs == lengthof(buf_id_array));
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/buffer/cached_buf.c b/src/backend/storage/buffer/cached_buf.c
new file mode 100644
index 0000000..dd4ae88
--- /dev/null
+++ b/src/backend/storage/buffer/cached_buf.c
@@ -0,0 +1,324 @@
+/*-------------------------------------------------------------------------
+ *
+ * cached_buf.c
+ *	  routines for mapping relations to the indexes of auxillary cached
+ *	  buffers.
+ *
+ * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ *	  src/backend/storage/buffer/cached_buf.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "storage/bufmgr.h"
+#include "storage/buf_internals.h"
+
+
+/*
+ * Each relation and its buffer information are cached in a hash table
+ * located in shared memory. These cached buffers are chained together
+ * in a doubly-linked list.
+ */
+static HTAB		*CachedBufHash = NULL;
+
+BufDlistEnt	*BufDlistEntArray = NULL;
+static CachedBufTableLock *CachedBufLockArray = NULL;
+static uint32 CachedBufTableHashCode(RelFileNode *rnode);
+static LWLock *GetCachedBufPartitionLock(RelFileNode *rnode,
+										 uint32 *hashcodePtr);
+
+/*
+ * CachedBufShmemSize
+ * 		Estimate space needed for mapping cached buffer hash table
+ *
+ * 		size of lookup table is the desired hash table size
+ *		(possibly more than NBuffers)
+ */
+Size
+CachedBufShmemSize(void)
+{
+	Size		size = 0;
+
+	/* size of cached buffer lookup table */
+	size = add_size(size, hash_estimate_size(NBuffers,
+											sizeof(CachedBufEnt)));
+
+	/* size of cached buffer dlist entry array */
+	size = add_size(size, mul_size(NBuffers, sizeof(BufDlistEnt)));
+
+	/* size of locks */
+	size = add_size(size, mul_size(NBuffers,
+								   sizeof(CachedBufTableLock)));
+
+	return size;
+}
+
+/*
+ * InitCachedBLockTable
+ *      Initialize the cached block hash table and related data
+ *      structures at shared memory initialization.
+ */
+void
+InitCachedBufTable(int size)
+{
+	HASHCTL		info;
+	bool		foundList, foundLock;
+	int			i;
+
+	info.keysize = sizeof(RelFileNode);
+	info.entrysize = sizeof(CachedBufEnt);
+	info.num_partitions = NUM_MAP_PARTITIONS;
+
+	CachedBufHash = ShmemInitHash("Cached Block Lookup Table",
+								  size, size,
+								  &info,
+								  HASH_ELEM | HASH_BLOBS |
+								  HASH_PARTITION);
+
+	BufDlistEntArray = ShmemInitStruct("dlist entry array",
+									   size * sizeof(BufDlistEnt),
+									   &foundList);
+
+	CachedBufLockArray = (CachedBufTableLock *)
+			ShmemInitStruct("partition lock",
+							size * sizeof(CachedBufTableLock),
+							&foundLock);
+
+	if (!foundList && !foundLock)
+	{
+		CachedBufLockArray->cacheTrancheId = LWLockNewTrancheId();
+
+		for (i = 0; i < NUM_MAP_PARTITIONS; i++)
+			LWLockInitialize(&CachedBufLockArray->cacheLock[i],
+							 CachedBufLockArray->cacheTrancheId);
+	}
+	LWLockRegisterTranche(CachedBufLockArray->cacheTrancheId,
+						  "cached_block_tranche_id");
+}
+
+ /*
+  * CachedBufTableHashCode
+  *      Compute the hash code associated with target relation
+  *
+  * This must be passed to the insert/lookup/delete routines along with the
+  * relation. We do it like this because the callers need to know the hash
+  * code in order to determine which partition to lock, and we don't want
+  * to do the hash computation twice (hash_any is a bit slow).
+  */
+static uint32
+CachedBufTableHashCode(RelFileNode *rnode)
+{
+	return get_hash_value(CachedBufHash, (void *) rnode);
+}
+
+/*
+ * GetCachedBufPartitionLock
+ * 		Get lock instance for partition of cached buffer lookup table
+ */
+static LWLock *
+GetCachedBufPartitionLock(RelFileNode *rnode, uint32 *hashcode)
+{
+	*hashcode = CachedBufTableHashCode(rnode);
+
+	return &CachedBufLockArray->cacheLock[*hashcode % NUM_MAP_PARTITIONS];
+}
+
+/*
+ * CachedBufTableInsert
+ *		Insert a hash table entry for given relation and buffer ID
+ *
+ * If a hash entry for the target relation already exists, only buffer ID
+ * is inserted. Chain the buffer ID to the doubly-linked list.
+ *
+ * This function must hold exclusive lock for relation's partition.
+ */
+void
+CachedBufTableInsert(RelFileNode *rnode, int buf_id)
+{
+	uint32			hashcode;
+	LWLock			*map_lock;
+	CachedBufEnt	*hash_entry;
+	bool			found;
+
+	map_lock = GetCachedBufPartitionLock(rnode, &hashcode);
+	LWLockAcquire(map_lock, LW_EXCLUSIVE);
+
+	/* look up or create a hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) rnode,
+									hashcode,
+									HASH_ENTER,
+									&found);
+
+	/* If not found, initialize linked list */
+	if (!found)
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+
+	Assert(buf_id >= 0);
+	cb_dlist_push_head(hash_entry, buf_id);
+
+	LWLockRelease(map_lock);
+}
+
+/*
+ * CachedBufLookup
+ *		Lookup the buffers of target relation in the cached buffer hash
+ *      table, insert the buffer IDs to the given array of buffer ID, and
+ *      return the number of buffers inserted in the array.
+ *
+ * This function must hold shared LWLock for relation's partition.
+ */
+int
+CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				  int *forknum_indexes, BlockNumber *firstDelBlock,
+				  int *buf_id_array, int size)
+{
+	uint32		hashcode;
+	LWLock		*map_lock;
+	CachedBufEnt	*hash_entry;
+	CachedBufEnt	temp_head_entry;
+	BufDlistEnt	*curr_entry = NULL;
+	int		i;
+	int		curr_buf_id;
+	int		new_curr_buf_id;
+	int		count = 0;
+	bool	target_buf = false;
+
+	map_lock = GetCachedBufPartitionLock(&rnode, &hashcode);
+	LWLockAcquire(map_lock, LW_SHARED);
+
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &rnode,
+									hashcode,
+									HASH_FIND,
+									NULL);
+
+	if (!hash_entry)
+	{
+		LWLockRelease(map_lock);
+		return 0;	/* No existing hash entry */
+	}
+
+	/* Initial temporary dlist */
+	temp_head_entry.head = CACHED_BUF_END_OF_LIST;
+
+	/* When traversing the main dlist, start from head */
+	curr_buf_id = hash_entry->head;
+
+	while(curr_buf_id != CACHED_BUF_END_OF_LIST && count < size)
+	{
+		BufferDesc *bufHdr = GetBufferDescriptor(curr_buf_id);
+		curr_entry = cb_dlist_entry(curr_buf_id);
+		new_curr_buf_id = curr_entry->next;
+
+		/* Check if it's our target buffer */
+		for (i = 0; i < nforks; i++)
+		{
+			if (bufHdr->tag.forkNum != forkNum[i])
+				continue;
+			else
+			{
+				if (bufHdr->tag.blockNum >= firstDelBlock[i])
+					target_buf = true;
+				break;
+			}
+		}
+
+		if (target_buf)
+		{
+			forknum_indexes[count] = i;
+			buf_id_array[count] = curr_buf_id;
+			++count;
+		}
+		else
+		{
+			/*
+			 * It's not the target buffer. Remove the current buffer ID
+			 * from the current list of target buffer IDs and store it
+			 * to a temporary list.
+			 */
+			cb_dlist_delete(hash_entry, curr_buf_id);
+			cb_dlist_push_head(&temp_head_entry, curr_buf_id);
+		}
+		/* Move current pointer to next */
+		curr_buf_id = new_curr_buf_id;
+	}
+
+	/* Check if main dlist is now empty */
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		hash_entry->head = temp_head_entry.head;
+		temp_head_entry.head = CACHED_BUF_END_OF_LIST;
+	}
+
+	/* If we have a temporary dlist, append it to the main dlist */
+	if (!cb_dlist_is_empty(hash_entry) &&
+		!cb_dlist_is_empty(&temp_head_entry))
+		cb_dlist_combine(hash_entry, &temp_head_entry);
+
+	LWLockRelease(map_lock);
+
+	return count;
+}
+
+/*
+ * CachedBufTableDelete
+ *		Unlink the buffer ID from the doubly-linked list, then remove
+ *		the relation hash entry if its list is empty.
+ *
+ * This function must hold exclusive lock for relation's partition.
+ */
+void
+CachedBufTableDelete(RelFileNode *rnode, int buf_id)
+{
+	LWLock		*map_lock;
+	uint32		hashcode;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	Assert(buf_id >= 0);
+
+	map_lock = GetCachedBufPartitionLock(rnode, &hashcode);
+	LWLockAcquire(map_lock, LW_EXCLUSIVE);
+
+	/* look up hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) rnode,
+									hashcode,
+									HASH_FIND,
+									&found);
+
+	if (!found)		/* rnode not found, nothing to do */
+	{
+		LWLockRelease(map_lock);
+		return;
+	}
+
+	cb_dlist_delete(hash_entry, buf_id);
+
+	/*
+	 * If there's no more cached elements for the target relation,
+	 * remove the relation hash entry.
+	 */
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) rnode,
+										hashcode,
+										HASH_REMOVE,
+										NULL);
+
+		if (!hash_entry)
+			elog(ERROR, "cached buffer hash table corrupted");
+	}
+
+	LWLockRelease(map_lock);
+}
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index 6ffe184..b9f7f26 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -297,6 +297,7 @@ typedef struct CkptSortItem
 
 extern CkptSortItem *CkptBufferIds;
 
+
 /*
  * Internal buffer management routines
  */
@@ -338,4 +339,154 @@ extern void DropRelFileNodeLocalBuffers(RelFileNode rnode, ForkNumber forkNum,
 extern void DropRelFileNodeAllLocalBuffers(RelFileNode rnode);
 extern void AtEOXact_LocalBuffers(bool isCommit);
 
+/* in cached_buf.c */
+
+/* entry to the cached buffer hash table */
+typedef struct CachedBufEnt
+{
+	RelFileNode		rnode;
+	int				head;	/* head index of list */
+} CachedBufEnt;
+
+/* entry to the doubly-linked list */
+typedef struct BufDlistEnt
+{
+	int		prev;
+	int		next;
+} BufDlistEnt;
+
+#define NUM_MAP_PARTITIONS	128
+#define BUF_ID_ARRAY_SIZE	100
+
+/* end of list entry for cached buffer list */
+#define CACHED_BUF_END_OF_LIST	(-1)
+
+extern BufDlistEnt	*BufDlistEntArray;
+
+/* lock for cached buffer hash table */
+typedef struct CachedBufTableLock
+{
+	LWLock	cacheLock[NUM_MAP_PARTITIONS];
+	int		cacheTrancheId;
+} CachedBufTableLock;
+
+extern Size CachedBufShmemSize(void);
+extern void InitCachedBufTable(int size);
+extern void CachedBufTableInsert(RelFileNode *rnode, int buf_id);
+extern int CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				  int *forknum_indexes, BlockNumber *firstDelBlock,
+				  int *buf_id_array, int size);
+extern void CachedBufTableDelete(RelFileNode *rnode, int buf_id);
+
+/*
+ * inline functions for the doubly-linked list of cached buffers
+ */
+/* Return the dlist entry */
+static inline BufDlistEnt *
+cb_dlist_entry(int buf_id)
+{
+	return BufDlistEntArray + buf_id;
+}
+
+/* Return the cached buffer id of the target entry */
+static inline int
+cb_dlist_buf_id(BufDlistEnt *entry)
+{
+	return entry - BufDlistEntArray;
+}
+
+/* Return the next entry */
+static inline BufDlistEnt *
+cb_dlist_next(BufDlistEnt *entry)
+{
+	return entry->next == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->next);
+}
+
+/* Return the prev entry */
+static inline BufDlistEnt *
+cb_dlist_prev(BufDlistEnt *entry)
+{
+	return entry->prev == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->prev);
+}
+
+/* Return if dlist is empty */
+static inline bool
+cb_dlist_is_empty(CachedBufEnt *hash_entry)
+{
+	return hash_entry->head == CACHED_BUF_END_OF_LIST;
+}
+
+/* Push to head of dlist */
+static inline void
+cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt *new_entry = cb_dlist_entry(buf_id);
+
+	new_entry->prev = CACHED_BUF_END_OF_LIST;
+	new_entry->next = hash_entry->head;
+
+	if (!cb_dlist_is_empty(hash_entry))
+		cb_dlist_next(new_entry)->prev = buf_id;
+
+	hash_entry->head = buf_id;
+}
+
+/* Remove the buffer ID from dlist */
+static inline void
+cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*curr_entry = cb_dlist_entry(buf_id);
+	BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+	BufDlistEnt	*next_entry = cb_dlist_next(curr_entry);
+	BufDlistEnt	*prev_entry = cb_dlist_prev(curr_entry);
+
+	/*
+	 * If there is a next entry, update its prev field.
+	 * Otherwise, entry is currently at tail.
+	 */
+	if (next_entry != NULL)
+		next_entry->prev = curr_entry->prev;
+	else
+	{
+		/* If tail is also the head entry */
+		if (curr_entry == head_entry)
+		{
+			hash_entry->head = CACHED_BUF_END_OF_LIST;
+			return;
+		}
+	}
+
+	/*
+	 * If there is a previous entry, update its next field.
+	 * Otherwise, entry is at head so update to new head.
+	 */
+	if (prev_entry != NULL)
+		prev_entry->next = curr_entry->next;
+	else
+		hash_entry->head = curr_entry->next;
+}
+
+/* Append head of temporary dlist to main dlist */
+static inline void
+cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp)
+{
+	BufDlistEnt	*curr_main = cb_dlist_entry(main->head);
+	BufDlistEnt  *curr_temp = cb_dlist_entry(temp->head);
+	int 	tail;
+
+	/* Point to the tail of main dlist */
+	while (curr_main->next != CACHED_BUF_END_OF_LIST)
+		curr_main = cb_dlist_next(curr_main);
+	tail = cb_dlist_buf_id(curr_main);
+
+	/* Append the temporary dlist to tail of main dlist */
+	curr_main->next = temp->head;
+	curr_temp->prev = tail;
+
+	/* Clear the head of temporary dlist */
+	temp->head = CACHED_BUF_END_OF_LIST;
+}
+
 #endif							/* BUFMGR_INTERNALS_H */
-- 
1.8.3.1

#2k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#1)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi,

Another one that I'd need feedback of is the use of new dlist operations

for this cached buffer list. I did not use in this patch the existing

Postgres dlist architecture (ilist.h) because I want to save memory space

as much as possible especially when NBuffers become large. Both dlist_node

& dlist_head are 16 bytes. OTOH, two int pointers for this patch is 8 bytes.

In cb_dlist_combine(), the code block below can impact performance
especially for cases when the doubly linked list is long (IOW, many cached buffers).
/* Point to the tail of main dlist */
while (curr_main->next != CACHEDBLOCK_END_OF_LIST)
curr_main = cb_dlist_next(curr_main);

Attached is an improved version of the previous patch, which adds a pointer
information of the TAIL field in order to speed up the abovementioned operation.
I stored the tail field in the prev pointer of the head entry (maybe not a typical
approach). A more typical one is by adding a tail field (int tail) to CachedBufferEnt,
but I didn’t do that because as I mentioned in previous email I want to avoid
using more memory as much as possible.
The patch worked as intended and passed the tests.

Any thoughts?

Regards,
Kirk Jamison

Attachments:

v2-Optimize-dropping-of-relation-buffers-using-dlist.patchapplication/octet-stream; name=v2-Optimize-dropping-of-relation-buffers-using-dlist.patchDownload
From 8e1c614a645e2b688cf0166d119e903050bd5fe2 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 25 Sep 2019 02:24:13 +0000
Subject: [PATCH] Optimize dropping of relation buffers using dlist

---
 src/backend/storage/buffer/Makefile     |   2 +-
 src/backend/storage/buffer/buf_init.c   |   6 +
 src/backend/storage/buffer/bufmgr.c     |  84 +++++----
 src/backend/storage/buffer/cached_buf.c | 324 ++++++++++++++++++++++++++++++++
 src/include/storage/buf_internals.h     | 163 ++++++++++++++++
 5 files changed, 539 insertions(+), 40 deletions(-)
 create mode 100644 src/backend/storage/buffer/cached_buf.c

diff --git a/src/backend/storage/buffer/Makefile b/src/backend/storage/buffer/Makefile
index 2c10fba..fa23c0c 100644
--- a/src/backend/storage/buffer/Makefile
+++ b/src/backend/storage/buffer/Makefile
@@ -12,6 +12,6 @@ subdir = src/backend/storage/buffer
 top_builddir = ../../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = buf_table.o buf_init.o bufmgr.o freelist.o localbuf.o
+OBJS = buf_table.o buf_init.o bufmgr.o freelist.o localbuf.o cached_buf.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/storage/buffer/buf_init.c b/src/backend/storage/buffer/buf_init.c
index ccd2c31..3e4eb40 100644
--- a/src/backend/storage/buffer/buf_init.c
+++ b/src/backend/storage/buffer/buf_init.c
@@ -146,6 +146,9 @@ InitBufferPool(void)
 	/* Init other shared buffer-management stuff */
 	StrategyInitialize(!foundDescs);
 
+	/* Init cached buffer hash table and related shmem data structures */
+	InitCachedBufTable(NBuffers);
+
 	/* Initialize per-backend file flush context */
 	WritebackContextInit(&BackendWritebackContext,
 						 &backend_flush_after);
@@ -189,5 +192,8 @@ BufferShmemSize(void)
 	/* size of checkpoint sort array in bufmgr.c */
 	size = add_size(size, mul_size(NBuffers, sizeof(CkptSortItem)));
 
+	/* size of cached buffer shmem data structures */
+	size = add_size(size, CachedBufShmemSize());
+
 	return size;
 }
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 483f705..0ec6a78 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -1268,6 +1268,8 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 			return buf;
 		}
+		/* Insert an entry into the cached block list */
+		CachedBufTableInsert(&newTag.rnode, buf->buf_id);
 
 		/*
 		 * Need to lock the buffer header too in order to change its tag.
@@ -1285,6 +1287,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 			break;
 
 		UnlockBufHdr(buf, buf_state);
+		CachedBufTableDelete(&newTag.rnode, buf->buf_id);
 		BufTableDelete(&newTag, newHash);
 		if (oldPartitionLock != NULL &&
 			oldPartitionLock != newPartitionLock)
@@ -1319,6 +1322,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	if (oldPartitionLock != NULL)
 	{
+		CachedBufTableDelete(&oldTag.rnode, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
 		if (oldPartitionLock != newPartitionLock)
 			LWLockRelease(oldPartitionLock);
@@ -1432,7 +1436,10 @@ retry:
 	 * Remove the buffer from the lookup hashtable, if it was in there.
 	 */
 	if (oldFlags & BM_TAG_VALID)
+	{
+		CachedBufTableDelete(&oldTag.rnode, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
+	}
 
 	/*
 	 * Done with mapping lock.
@@ -2916,19 +2923,15 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
 DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
-	int			i;
-	int			j;
+	int			i, j, nbufs;
+	int			buf_id_array[BUF_ID_ARRAY_SIZE];
+	int			forknum_indexes[BUF_ID_ARRAY_SIZE];
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2942,45 +2945,48 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	do
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		nbufs = CachedBufLookup(rnode.node, forkNum, nforks,
+								forknum_indexes, firstDelBlock,
+								buf_id_array, lengthof(buf_id_array));
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+		for (i = 0; i < nbufs; i++)
+		{
+			BufferDesc *bufHdr = GetBufferDescriptor(buf_id_array[i]);
+			uint32	buf_state;
+			int		index = forknum_indexes[i];
 
-		buf_state = LockBufHdr(bufHdr);
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag
+			 * before we attempt to lock the buffer; this saves a lot of
+			 * lock acquisitions in typical cases.  It should be safe
+			 * because the caller must have AccessExclusiveLock on the
+			 * relation, or some other reason to be certain that no one is
+			 * loading new pages of the rel into the buffer pool.
+			 * (Otherwise we might well miss such pages entirely.)
+			 * Therefore, while the tag might be changing while we look at
+			 * it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible,
+			 * and false positives are safe because we'll recheck after
+			 * getting the buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but
+			 * the incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
+
+			buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
-		{
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
-			{
+				bufHdr->tag.forkNum == forkNum[index] &&
+				bufHdr->tag.blockNum >= firstDelBlock[index])
 				InvalidateBuffer(bufHdr); /* releases spinlock */
-				break;
-			}
+			else
+				UnlockBufHdr(bufHdr, buf_state);
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
-	}
+	} while (nbufs == lengthof(buf_id_array));
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/buffer/cached_buf.c b/src/backend/storage/buffer/cached_buf.c
new file mode 100644
index 0000000..3a3b1fb
--- /dev/null
+++ b/src/backend/storage/buffer/cached_buf.c
@@ -0,0 +1,324 @@
+/*-------------------------------------------------------------------------
+ *
+ * cached_buf.c
+ *	  routines for mapping relations to the indexes of auxillary cached
+ *	  buffers.
+ *
+ * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ *	  src/backend/storage/buffer/cached_buf.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "storage/bufmgr.h"
+#include "storage/buf_internals.h"
+
+
+/*
+ * Each relation and its buffer information are cached in a hash table
+ * located in shared memory. These cached buffers are chained together
+ * in a doubly-linked list.
+ */
+static HTAB		*CachedBufHash = NULL;
+
+BufDlistEnt	*BufDlistEntArray = NULL;
+static CachedBufTableLock *CachedBufLockArray = NULL;
+static uint32 CachedBufTableHashCode(RelFileNode *rnode);
+static LWLock *GetCachedBufPartitionLock(RelFileNode *rnode,
+										 uint32 *hashcodePtr);
+
+/*
+ * CachedBufShmemSize
+ * 		Estimate space needed for mapping cached buffer hash table
+ *
+ * 		size of lookup table is the desired hash table size
+ *		(possibly more than NBuffers)
+ */
+Size
+CachedBufShmemSize(void)
+{
+	Size		size = 0;
+
+	/* size of cached buffer lookup table */
+	size = add_size(size, hash_estimate_size(NBuffers,
+											sizeof(CachedBufEnt)));
+
+	/* size of cached buffer dlist entry array */
+	size = add_size(size, mul_size(NBuffers, sizeof(BufDlistEnt)));
+
+	/* size of locks */
+	size = add_size(size, mul_size(NBuffers,
+								   sizeof(CachedBufTableLock)));
+
+	return size;
+}
+
+/*
+ * InitCachedBLockTable
+ *      Initialize the cached block hash table and related data
+ *      structures at shared memory initialization.
+ */
+void
+InitCachedBufTable(int size)
+{
+	HASHCTL		info;
+	bool		foundList, foundLock;
+	int			i;
+
+	info.keysize = sizeof(RelFileNode);
+	info.entrysize = sizeof(CachedBufEnt);
+	info.num_partitions = NUM_MAP_PARTITIONS;
+
+	CachedBufHash = ShmemInitHash("Cached Block Lookup Table",
+								  size, size,
+								  &info,
+								  HASH_ELEM | HASH_BLOBS |
+								  HASH_PARTITION);
+
+	BufDlistEntArray = ShmemInitStruct("dlist entry array",
+									   size * sizeof(BufDlistEnt),
+									   &foundList);
+
+	CachedBufLockArray = (CachedBufTableLock *)
+			ShmemInitStruct("partition lock",
+							size * sizeof(CachedBufTableLock),
+							&foundLock);
+
+	if (!foundList && !foundLock)
+	{
+		CachedBufLockArray->cacheTrancheId = LWLockNewTrancheId();
+
+		for (i = 0; i < NUM_MAP_PARTITIONS; i++)
+			LWLockInitialize(&CachedBufLockArray->cacheLock[i],
+							 CachedBufLockArray->cacheTrancheId);
+	}
+	LWLockRegisterTranche(CachedBufLockArray->cacheTrancheId,
+						  "cached_block_tranche_id");
+}
+
+ /*
+  * CachedBufTableHashCode
+  *      Compute the hash code associated with target relation
+  *
+  * This must be passed to the insert/lookup/delete routines along with the
+  * relation. We do it like this because the callers need to know the hash
+  * code in order to determine which partition to lock, and we don't want
+  * to do the hash computation twice (hash_any is a bit slow).
+  */
+static uint32
+CachedBufTableHashCode(RelFileNode *rnode)
+{
+	return get_hash_value(CachedBufHash, (void *) rnode);
+}
+
+/*
+ * GetCachedBufPartitionLock
+ * 		Get lock instance for partition of cached buffer lookup table
+ */
+static LWLock *
+GetCachedBufPartitionLock(RelFileNode *rnode, uint32 *hashcode)
+{
+	*hashcode = CachedBufTableHashCode(rnode);
+
+	return &CachedBufLockArray->cacheLock[*hashcode % NUM_MAP_PARTITIONS];
+}
+
+/*
+ * CachedBufTableInsert
+ *		Insert a hash table entry for given relation and buffer ID
+ *
+ * If a hash entry for the target relation already exists, only buffer ID
+ * is inserted. Chain the buffer ID to the doubly-linked list.
+ *
+ * This function must hold exclusive lock for relation's partition.
+ */
+void
+CachedBufTableInsert(RelFileNode *rnode, int buf_id)
+{
+	uint32			hashcode;
+	LWLock			*map_lock;
+	CachedBufEnt	*hash_entry;
+	bool			found;
+
+	map_lock = GetCachedBufPartitionLock(rnode, &hashcode);
+	LWLockAcquire(map_lock, LW_EXCLUSIVE);
+
+	/* look up or create a hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) rnode,
+									hashcode,
+									HASH_ENTER,
+									&found);
+
+	/* If not found, initialize linked list */
+	if (!found)
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+
+	Assert(buf_id >= 0);
+	cb_dlist_push_head(hash_entry, buf_id);
+
+	LWLockRelease(map_lock);
+}
+
+/*
+ * CachedBufLookup
+ *		Lookup the buffers of target relation in the cached buffer hash
+ *      table, insert the buffer IDs to the given array of buffer ID, and
+ *      return the number of buffers inserted in the array.
+ *
+ * This function must hold shared LWLock for relation's partition.
+ */
+int
+CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				  int *forknum_indexes, BlockNumber *firstDelBlock,
+				  int *buf_id_array, int size)
+{
+	uint32		hashcode;
+	LWLock		*map_lock;
+	CachedBufEnt	*hash_entry;
+	CachedBufEnt	temp_head_entry;
+	BufDlistEnt	*curr_entry = NULL;
+	int		i;
+	int		curr_buf_id;
+	int		new_curr_buf_id;
+	int		count = 0;
+	bool		target_buf = false;
+
+	map_lock = GetCachedBufPartitionLock(&rnode, &hashcode);
+	LWLockAcquire(map_lock, LW_SHARED);
+
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &rnode,
+									hashcode,
+									HASH_FIND,
+									NULL);
+
+	if (!hash_entry)
+	{
+		LWLockRelease(map_lock);
+		return 0;	/* No existing hash entry */
+	}
+
+	/* Initial temporary dlist */
+	temp_head_entry.head = CACHED_BUF_END_OF_LIST;
+
+	/* When traversing the main dlist, start from head */
+	curr_buf_id = hash_entry->head;
+
+	while(curr_buf_id != CACHED_BUF_END_OF_LIST && count < size)
+	{
+		BufferDesc *bufHdr = GetBufferDescriptor(curr_buf_id);
+		curr_entry = cb_dlist_entry(curr_buf_id);
+		new_curr_buf_id = curr_entry->next;
+
+		/* Check if it's our target buffer */
+		for (i = 0; i < nforks; i++)
+		{
+			if (bufHdr->tag.forkNum != forkNum[i])
+				continue;
+			else
+			{
+				if (bufHdr->tag.blockNum >= firstDelBlock[i])
+					target_buf = true;
+				break;
+			}
+		}
+
+		if (target_buf)
+		{
+			forknum_indexes[count] = i;
+			buf_id_array[count] = curr_buf_id;
+			++count;
+		}
+		else
+		{
+			/*
+			 * It's not the target buffer. Remove the current buffer ID
+			 * from the current list of target buffer IDs and store it
+			 * to a temporary list.
+			 */
+			cb_dlist_delete(hash_entry, curr_buf_id);
+			cb_dlist_push_head(&temp_head_entry, curr_buf_id);
+		}
+		/* Move current pointer to next */
+		curr_buf_id = new_curr_buf_id;
+	}
+
+	/* Check if main dlist is now empty */
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		hash_entry->head = temp_head_entry.head;
+		temp_head_entry.head = CACHED_BUF_END_OF_LIST;
+	}
+
+	/* If we have a temporary dlist, append it to the main dlist */
+	if (!cb_dlist_is_empty(hash_entry) &&
+		!cb_dlist_is_empty(&temp_head_entry))
+		cb_dlist_combine(hash_entry, &temp_head_entry);
+
+	LWLockRelease(map_lock);
+
+	return count;
+}
+
+/*
+ * CachedBufTableDelete
+ *		Unlink the buffer ID from the doubly-linked list, then remove
+ *		the relation hash entry if its list is empty.
+ *
+ * This function must hold exclusive lock for relation's partition.
+ */
+void
+CachedBufTableDelete(RelFileNode *rnode, int buf_id)
+{
+	LWLock		*map_lock;
+	uint32		hashcode;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	Assert(buf_id >= 0);
+
+	map_lock = GetCachedBufPartitionLock(rnode, &hashcode);
+	LWLockAcquire(map_lock, LW_EXCLUSIVE);
+
+	/* look up hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) rnode,
+									hashcode,
+									HASH_FIND,
+									&found);
+
+	if (!found)		/* rnode not found, nothing to do */
+	{
+		LWLockRelease(map_lock);
+		return;
+	}
+
+	cb_dlist_delete(hash_entry, buf_id);
+
+	/*
+	 * If there's no more cached elements for the target relation,
+	 * remove the relation hash entry.
+	 */
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) rnode,
+										hashcode,
+										HASH_REMOVE,
+										NULL);
+
+		if (!hash_entry)
+			elog(ERROR, "cached buffer hash table corrupted");
+	}
+
+	LWLockRelease(map_lock);
+}
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index 6ffe184..1efb41a 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -297,6 +297,7 @@ typedef struct CkptSortItem
 
 extern CkptSortItem *CkptBufferIds;
 
+
 /*
  * Internal buffer management routines
  */
@@ -338,4 +339,166 @@ extern void DropRelFileNodeLocalBuffers(RelFileNode rnode, ForkNumber forkNum,
 extern void DropRelFileNodeAllLocalBuffers(RelFileNode rnode);
 extern void AtEOXact_LocalBuffers(bool isCommit);
 
+/* in cached_buf.c */
+
+/* entry to the cached buffer hash table */
+typedef struct CachedBufEnt
+{
+	RelFileNode		rnode;
+	int			head;	/* head index of list */
+} CachedBufEnt;
+
+/* entry to the doubly-linked list */
+typedef struct BufDlistEnt
+{
+	int		prev;
+	int		next;
+} BufDlistEnt;
+
+#define NUM_MAP_PARTITIONS	128
+#define BUF_ID_ARRAY_SIZE	100
+
+/* end of list entry for cached buffer list */
+#define CACHED_BUF_END_OF_LIST	(-1)
+
+extern BufDlistEnt	*BufDlistEntArray;
+
+/* lock for cached buffer hash table */
+typedef struct CachedBufTableLock
+{
+	LWLock	cacheLock[NUM_MAP_PARTITIONS];
+	int		cacheTrancheId;
+} CachedBufTableLock;
+
+extern Size CachedBufShmemSize(void);
+extern void InitCachedBufTable(int size);
+extern void CachedBufTableInsert(RelFileNode *rnode, int buf_id);
+extern int CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				  int *forknum_indexes, BlockNumber *firstDelBlock,
+				  int *buf_id_array, int size);
+extern void CachedBufTableDelete(RelFileNode *rnode, int buf_id);
+
+/*
+ * inline functions for the doubly-linked list of cached buffers
+ */
+/* Return the dlist entry */
+static inline BufDlistEnt *
+cb_dlist_entry(int buf_id)
+{
+	return BufDlistEntArray + buf_id;
+}
+
+/* Return the cached buffer id of the target entry */
+static inline int
+cb_dlist_buf_id(BufDlistEnt *entry)
+{
+	return entry - BufDlistEntArray;
+}
+
+/* Return the next entry */
+static inline BufDlistEnt *
+cb_dlist_next(BufDlistEnt *entry)
+{
+	return entry->next == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->next);
+}
+
+/* Return the prev entry */
+static inline BufDlistEnt *
+cb_dlist_prev(BufDlistEnt *entry)
+{
+	return entry->prev == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->prev);
+}
+
+/* Return if dlist is empty */
+static inline bool
+cb_dlist_is_empty(CachedBufEnt *hash_entry)
+{
+	return hash_entry->head == CACHED_BUF_END_OF_LIST;
+}
+
+/* Push to head of dlist */
+static inline void
+cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt *new_entry = cb_dlist_entry(buf_id);
+
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		new_entry->prev = buf_id; /* TAIL */
+		new_entry->next = hash_entry->head;
+	}
+	else
+	{
+		BufDlistEnt *head_entry = cb_dlist_entry(hash_entry->head);
+		int tail = head_entry->prev;
+		new_entry->prev = tail;
+		new_entry->next = hash_entry->head;
+		cb_dlist_next(new_entry)->prev = buf_id;
+	}
+	hash_entry->head = buf_id;
+}
+
+/* Remove the buffer ID from dlist */
+static inline void
+cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*curr_entry = cb_dlist_entry(buf_id);
+	BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+	BufDlistEnt	*next_entry = cb_dlist_next(curr_entry);
+	BufDlistEnt	*prev_entry = cb_dlist_prev(curr_entry);
+
+	int tail = head_entry->prev;
+	BufDlistEnt	*tail_entry = cb_dlist_entry(tail);
+
+	/* If entry to be deleted is the only entry */
+	if (head_entry == tail_entry)
+	{
+		head_entry->prev = CACHED_BUF_END_OF_LIST;
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+		return;
+	}
+
+	/*
+	 * If there is a next entry, update its prev field.
+	 * Otherwise, current entry is at tail (but not at head),
+	 * so update the new tail.
+	 */
+	if (next_entry != NULL)
+		next_entry->prev = curr_entry->prev;
+	else
+		head_entry->prev = tail;
+
+	/*
+	 * If the previous entry is not the tail entry, update its
+	 * next field. Otherwise, current entry is at head (but not
+	 * at tail). Update the new head entry and its tail pointer.
+	 */
+	if (prev_entry != tail_entry)
+		prev_entry->next = curr_entry->next;
+	else
+		hash_entry->head = curr_entry->next;
+}
+
+/* Append head of temporary dlist to main dlist */
+static inline void
+cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp)
+{
+	BufDlistEnt	*main_head_entry = cb_dlist_entry(main->head);
+	BufDlistEnt  *temp_head_entry = cb_dlist_entry(temp->head);
+	int main_tail = main_head_entry->prev;
+	int temp_tail = temp_head_entry->prev;
+	BufDlistEnt	*main_tail_entry = cb_dlist_entry(main_tail);
+
+	/* Append the temporary dlist to main dlist */
+	main_tail_entry->next = temp->head;
+	temp_head_entry->prev = main_tail;
+	main_head_entry->prev = temp_tail;
+
+	/* Clear the head of temporary dlist */
+	temp_head_entry->prev = CACHED_BUF_END_OF_LIST;
+	temp->head = CACHED_BUF_END_OF_LIST;
+}
+
 #endif							/* BUFMGR_INTERNALS_H */
-- 
1.8.3.1

#3Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: k.jamison@fujitsu.com (#2)
Re: [Patch] Optimize dropping of relation buffers using dlist

Hi Kirk,

On Tue, Nov 05, 2019 at 09:58:22AM +0000, k.jamison@fujitsu.com wrote:

Hi,

Another one that I'd need feedback of is the use of new dlist operations

for this cached buffer list. I did not use in this patch the existing

Postgres dlist architecture (ilist.h) because I want to save memory space

as much as possible especially when NBuffers become large. Both dlist_node

& dlist_head are 16 bytes. OTOH, two int pointers for this patch is 8 bytes.

In cb_dlist_combine(), the code block below can impact performance
especially for cases when the doubly linked list is long (IOW, many cached buffers).
/* Point to the tail of main dlist */
while (curr_main->next != CACHEDBLOCK_END_OF_LIST)
curr_main = cb_dlist_next(curr_main);

Attached is an improved version of the previous patch, which adds a pointer
information of the TAIL field in order to speed up the abovementioned operation.
I stored the tail field in the prev pointer of the head entry (maybe not a typical
approach). A more typical one is by adding a tail field (int tail) to CachedBufferEnt,
but I didn’t do that because as I mentioned in previous email I want to avoid
using more memory as much as possible.
The patch worked as intended and passed the tests.

Any thoughts?

A couple of comments based on briefly looking at the patch.

1) I don't think you should / need to expose most of the ne stuff in
buf_internals.h. It's only used from buf_internals.c and having all
the various cb_dlist_* function in .h seems strange.

2) This adds another hashtable maintenance to BufferAlloc etc. but
you've only done tests / benchmark for the case this optimizes. I
think we need to see a benchmark for workload that allocates and
invalidates lot of buffers. A pgbench with a workload that fits into
RAM but not into shared buffers would be interesting.

3) I see this triggered a failure on cputube, in the commit_ts TAP test.
That's a bit strange, someone should investigate I guess.

https://travis-ci.org/postgresql-cfbot/postgresql/builds/607563900

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#4Robert Haas
robertmhaas@gmail.com
In reply to: Tomas Vondra (#3)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Nov 5, 2019 at 10:34 AM Tomas Vondra
<tomas.vondra@2ndquadrant.com> wrote:

2) This adds another hashtable maintenance to BufferAlloc etc. but
you've only done tests / benchmark for the case this optimizes. I
think we need to see a benchmark for workload that allocates and
invalidates lot of buffers. A pgbench with a workload that fits into
RAM but not into shared buffers would be interesting.

Yeah, it seems pretty hard to believe that this won't be bad for some
workloads. Not only do you have the overhead of the hash table
operations, but you also have locking overhead around that. A whole
new set of LWLocks where you have to take and release one of them
every time you allocate or invalidate a buffer seems likely to cause a
pretty substantial contention problem.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#5k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Robert Haas (#4)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thurs, November 7, 2019 1:27 AM (GMT+9), Robert Haas wrote:

On Tue, Nov 5, 2019 at 10:34 AM Tomas Vondra <tomas.vondra@2ndquadrant.com>
wrote:

2) This adds another hashtable maintenance to BufferAlloc etc. but
you've only done tests / benchmark for the case this optimizes. I
think we need to see a benchmark for workload that allocates and
invalidates lot of buffers. A pgbench with a workload that fits into
RAM but not into shared buffers would be interesting.

Yeah, it seems pretty hard to believe that this won't be bad for some workloads.
Not only do you have the overhead of the hash table operations, but you also
have locking overhead around that. A whole new set of LWLocks where you have
to take and release one of them every time you allocate or invalidate a buffer
seems likely to cause a pretty substantial contention problem.

I'm sorry for the late reply. Thank you Tomas and Robert for checking this patch.
Attached is the v3 of the patch.
- I moved the unnecessary items from buf_internals.h to cached_buf.c since most of
of those items are only used in that file.
- Fixed the bug of v2. Seems to pass both RT and TAP test now

Thanks for the advice on benchmark test. Please refer below for test and results.

[Machine spec]
CPU: 16, Number of cores per socket: 8
RHEL6.5, Memory: 240GB

scale: 3125 (about 46GB DB size)
shared_buffers = 8GB

[workload that fits into RAM but not into shared buffers]
pgbench -i -s 3125 cachetest
pgbench -c 16 -j 8 -T 600 cachetest

[Patched]
scaling factor: 3125
query mode: simple
number of clients: 16
number of threads: 8
duration: 600 s
number of transactions actually processed: 8815123
latency average = 1.089 ms
tps = 14691.436343 (including connections establishing)
tps = 14691.482714 (excluding connections establishing)

[Master/Unpatched]
...
number of transactions actually processed: 8852327
latency average = 1.084 ms
tps = 14753.814648 (including connections establishing)
tps = 14753.861589 (excluding connections establishing)

My patch caused a little overhead of about 0.42-0.46%, which I think is small.
Kindly let me know your opinions/comments about the patch or tests, etc.

Thanks,
Kirk Jamison

Attachments:

v3-Optimize-dropping-of-relation-buffers-using-dlist.patchapplication/octet-stream; name=v3-Optimize-dropping-of-relation-buffers-using-dlist.patchDownload
From f6c46678dd0191f803bc9f897a72f9fa5c1f9e27 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Tue, 12 Nov 2019 08:55:00 +0000
Subject: [PATCH] Optimize dropping of relation buffers using dlist

---
 src/backend/storage/buffer/Makefile     |   1 +
 src/backend/storage/buffer/buf_init.c   |   6 +
 src/backend/storage/buffer/bufmgr.c     |  84 +++---
 src/backend/storage/buffer/cached_buf.c | 450 ++++++++++++++++++++++++++++++++
 src/include/storage/buf_internals.h     |  12 +
 5 files changed, 514 insertions(+), 39 deletions(-)
 create mode 100644 src/backend/storage/buffer/cached_buf.c

diff --git a/src/backend/storage/buffer/Makefile b/src/backend/storage/buffer/Makefile
index fd7c40d..3662349 100644
--- a/src/backend/storage/buffer/Makefile
+++ b/src/backend/storage/buffer/Makefile
@@ -16,6 +16,7 @@ OBJS = \
 	buf_init.o \
 	buf_table.o \
 	bufmgr.o \
+	cached_buf.o \
 	freelist.o \
 	localbuf.o
 
diff --git a/src/backend/storage/buffer/buf_init.c b/src/backend/storage/buffer/buf_init.c
index ccd2c31..3e4eb40 100644
--- a/src/backend/storage/buffer/buf_init.c
+++ b/src/backend/storage/buffer/buf_init.c
@@ -146,6 +146,9 @@ InitBufferPool(void)
 	/* Init other shared buffer-management stuff */
 	StrategyInitialize(!foundDescs);
 
+	/* Init cached buffer hash table and related shmem data structures */
+	InitCachedBufTable(NBuffers);
+
 	/* Initialize per-backend file flush context */
 	WritebackContextInit(&BackendWritebackContext,
 						 &backend_flush_after);
@@ -189,5 +192,8 @@ BufferShmemSize(void)
 	/* size of checkpoint sort array in bufmgr.c */
 	size = add_size(size, mul_size(NBuffers, sizeof(CkptSortItem)));
 
+	/* size of cached buffer shmem data structures */
+	size = add_size(size, CachedBufShmemSize());
+
 	return size;
 }
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 7ad1073..6a5e3b6 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -1268,6 +1268,8 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 			return buf;
 		}
+		/* Insert an entry into the cached buffer list */
+		CachedBufTableInsert(&newTag.rnode, buf->buf_id);
 
 		/*
 		 * Need to lock the buffer header too in order to change its tag.
@@ -1285,6 +1287,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 			break;
 
 		UnlockBufHdr(buf, buf_state);
+		CachedBufTableDelete(&newTag.rnode, buf->buf_id);
 		BufTableDelete(&newTag, newHash);
 		if (oldPartitionLock != NULL &&
 			oldPartitionLock != newPartitionLock)
@@ -1319,6 +1322,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	if (oldPartitionLock != NULL)
 	{
+		CachedBufTableDelete(&oldTag.rnode, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
 		if (oldPartitionLock != newPartitionLock)
 			LWLockRelease(oldPartitionLock);
@@ -1432,7 +1436,10 @@ retry:
 	 * Remove the buffer from the lookup hashtable, if it was in there.
 	 */
 	if (oldFlags & BM_TAG_VALID)
+	{
+		CachedBufTableDelete(&oldTag.rnode, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
+	}
 
 	/*
 	 * Done with mapping lock.
@@ -2916,19 +2923,15 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
 DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
-	int			i;
-	int			j;
+	int			i, j, nbufs;
+	int			buf_id_array[BUF_ID_ARRAY_SIZE];
+	int			forknum_indexes[BUF_ID_ARRAY_SIZE];
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2942,45 +2945,48 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	do
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		nbufs = CachedBufLookup(rnode.node, forkNum, nforks,
+								forknum_indexes, firstDelBlock,
+								buf_id_array, lengthof(buf_id_array));
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+		for (i = 0; i < nbufs; i++)
+		{
+			BufferDesc	*bufHdr = GetBufferDescriptor(buf_id_array[i]);
+			uint32		buf_state;
+			int		index = forknum_indexes[i];
 
-		buf_state = LockBufHdr(bufHdr);
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag
+			 * before we attempt to lock the buffer; this saves a lot of
+			 * lock acquisitions in typical cases.  It should be safe
+			 * because the caller must have AccessExclusiveLock on the
+			 * relation, or some other reason to be certain that no one is
+			 * loading new pages of the rel into the buffer pool.
+			 * (Otherwise we might well miss such pages entirely.)
+			 * Therefore, while the tag might be changing while we look at
+			 * it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible,
+			 * and false positives are safe because we'll recheck after
+			 * getting the buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but
+			 * the incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
+
+			buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
-		{
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
-			{
+				bufHdr->tag.forkNum == forkNum[index] &&
+				bufHdr->tag.blockNum >= firstDelBlock[index])
 				InvalidateBuffer(bufHdr); /* releases spinlock */
-				break;
-			}
+			else
+				UnlockBufHdr(bufHdr, buf_state);
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
-	}
+	} while (nbufs == lengthof(buf_id_array));
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/buffer/cached_buf.c b/src/backend/storage/buffer/cached_buf.c
new file mode 100644
index 0000000..8d9f658
--- /dev/null
+++ b/src/backend/storage/buffer/cached_buf.c
@@ -0,0 +1,450 @@
+/*-------------------------------------------------------------------------
+ *
+ * cached_buf.c
+ *	  routines for mapping relations to the indexes of auxillary cached
+ *	  buffers.
+ *
+ * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ *	  src/backend/storage/buffer/cached_buf.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "storage/bufmgr.h"
+#include "storage/buf_internals.h"
+
+
+/*
+ * Each relation and its buffer information are cached in a hash table
+ * located in shared memory. These cached buffers are chained together
+ * in a doubly-linked list.
+ */
+#define NUM_MAP_PARTITIONS	128	/* number of partition of hash table */
+#define CACHED_BUF_END_OF_LIST	(-1) /* end of doubly linked list */
+
+/* entry to the cached buffer hash table */
+typedef struct CachedBufEnt
+{
+	RelFileNode		rnode;
+	int			head;	/* head index of list */
+} CachedBufEnt;
+
+/* entry to the doubly-linked list */
+typedef struct BufDlistEnt
+{
+	int		prev;
+	int		next;
+} BufDlistEnt;
+
+/* lock for cached buffer hash table */
+typedef struct CachedBufTableLock
+{
+	LWLock	cacheLock[NUM_MAP_PARTITIONS];
+	int	cacheTrancheId;
+} CachedBufTableLock;
+
+static HTAB *CachedBufHash = NULL;
+static BufDlistEnt *BufDlistEntArray = NULL;
+static CachedBufTableLock *CachedBufLockArray = NULL;
+static uint32 CachedBufTableHashCode(RelFileNode *rnode);
+static LWLock *GetCachedBufPartitionLock(RelFileNode *rnode,
+										 uint32 *hashcodePtr);
+static inline BufDlistEnt *cb_dlist_entry(int buf_id);
+static inline BufDlistEnt *cb_dlist_next(BufDlistEnt *entry);
+static inline BufDlistEnt *cb_dlist_prev(BufDlistEnt *entry);
+static inline bool cb_dlist_is_empty(CachedBufEnt *hash_entry);
+static inline void cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp);
+
+/*
+ * CachedBufShmemSize
+ * 		Estimate space needed for mapping cached buffer hash table
+ *
+ * 		size of lookup table is the desired hash table size
+ *		(possibly more than NBuffers)
+ */
+Size
+CachedBufShmemSize(void)
+{
+	Size		size = 0;
+
+	/* size of cached buffer lookup table */
+	size = add_size(size, hash_estimate_size(NBuffers,
+											 sizeof(CachedBufEnt)));
+
+	/* size of cached buffer dlist entry array */
+	size = add_size(size, mul_size(NBuffers, sizeof(BufDlistEnt)));
+
+	/* size of locks */
+	size = add_size(size, mul_size(NBuffers,
+								   sizeof(CachedBufTableLock)));
+
+	return size;
+}
+
+/*
+ * InitCachedBLockTable
+ *      Initialize the cached block hash table and related data
+ *      structures at shared memory initialization.
+ */
+void
+InitCachedBufTable(int size)
+{
+	HASHCTL		info;
+	bool		foundList, foundLock;
+	int		i;
+
+	info.keysize = sizeof(RelFileNode);
+	info.entrysize = sizeof(CachedBufEnt);
+	info.num_partitions = NUM_MAP_PARTITIONS;
+
+	CachedBufHash = ShmemInitHash("Cached Block Lookup Table",
+								  size, size, &info,
+								  HASH_ELEM | HASH_BLOBS |
+								  HASH_PARTITION);
+
+	BufDlistEntArray = ShmemInitStruct("dlist entry array",
+									   size * sizeof(BufDlistEnt),
+									   &foundList);
+
+	CachedBufLockArray = (CachedBufTableLock *)
+			ShmemInitStruct("partition lock",
+							size * sizeof(CachedBufTableLock),
+							&foundLock);
+
+	if (!foundList && !foundLock)
+	{
+		CachedBufLockArray->cacheTrancheId = LWLockNewTrancheId();
+
+		for (i = 0; i < NUM_MAP_PARTITIONS; i++)
+			LWLockInitialize(&CachedBufLockArray->cacheLock[i],
+							 CachedBufLockArray->cacheTrancheId);
+	}
+	LWLockRegisterTranche(CachedBufLockArray->cacheTrancheId,
+						  "cached_block_tranche_id");
+}
+
+ /*
+  * CachedBufTableHashCode
+  *      Compute the hash code associated with target relation
+  *
+  * This must be passed to the insert/lookup/delete routines along with the
+  * relation. We do it like this because the callers need to know the hash
+  * code in order to determine which partition to lock, and we don't want
+  * to do the hash computation twice (hash_any is a bit slow).
+  */
+static uint32
+CachedBufTableHashCode(RelFileNode *rnode)
+{
+	return get_hash_value(CachedBufHash, (void *) rnode);
+}
+
+/*
+ * GetCachedBufPartitionLock
+ * 		Get lock instance for partition of cached buffer lookup table
+ */
+static LWLock *
+GetCachedBufPartitionLock(RelFileNode *rnode, uint32 *hashcode)
+{
+	*hashcode = CachedBufTableHashCode(rnode);
+
+	return &CachedBufLockArray->cacheLock[*hashcode % NUM_MAP_PARTITIONS];
+}
+
+/*
+ * CachedBufTableInsert
+ *		Insert a hash table entry for given relation and buffer ID
+ *
+ * If a hash entry for the target relation already exists, only buffer ID
+ * is inserted. Chain the buffer ID to the doubly-linked list.
+ *
+ * This function must hold exclusive lock for relation's partition.
+ */
+void
+CachedBufTableInsert(RelFileNode *rnode, int buf_id)
+{
+	uint32		hashcode;
+	LWLock		*map_lock;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	map_lock = GetCachedBufPartitionLock(rnode, &hashcode);
+	LWLockAcquire(map_lock, LW_EXCLUSIVE);
+
+	/* look up or create a hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) rnode,
+									hashcode, HASH_ENTER, &found);
+
+	/* If not found, initialize linked list */
+	if (!found)
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+
+	cb_dlist_push_head(hash_entry, buf_id);
+
+	LWLockRelease(map_lock);
+}
+
+/*
+ * CachedBufLookup
+ *		Lookup the buffers of target relation in the cached buffer hash
+ *      table, insert the buffer IDs to the given array of buffer ID, and
+ *      return the number of buffers inserted in the array.
+ *
+ * This function must hold exclusive LWLock for relation's partition.
+ */
+int
+CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				  int *forknum_indexes, BlockNumber *firstDelBlock,
+				  int *buf_id_array, int size)
+{
+	uint32		hashcode;
+	LWLock		*map_lock;
+	CachedBufEnt	*hash_entry;
+	CachedBufEnt	temp_hash_entry;
+	BufDlistEnt	*curr_entry = NULL;
+	int		i;
+	int		curr_buf_id;
+	int		new_curr_buf_id;
+	int		count = 0;
+	bool		target_buf = false;
+
+	map_lock = GetCachedBufPartitionLock(&rnode, &hashcode);
+	LWLockAcquire(map_lock, LW_EXCLUSIVE);
+
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &rnode,
+									hashcode, HASH_FIND, NULL);
+
+	if (!hash_entry)
+	{
+		LWLockRelease(map_lock);
+		return 0;	/* No existing hash entry */
+	}
+
+	/* Initial temporary dlist */
+	temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+
+	/* When traversing the main dlist, start from head */
+	curr_buf_id = hash_entry->head;
+
+	while(curr_buf_id != CACHED_BUF_END_OF_LIST && count < size)
+	{
+		BufferDesc *bufHdr = GetBufferDescriptor(curr_buf_id);
+		curr_entry = BufDlistEntArray + curr_buf_id;
+		new_curr_buf_id = curr_entry->next;
+
+		/* Check if it's our target buffer */
+		for (i = 0; i < nforks; i++)
+		{
+			if (bufHdr->tag.forkNum != forkNum[i])
+				continue;
+			else
+			{
+				if (bufHdr->tag.blockNum >= firstDelBlock[i])
+					target_buf = true;
+				break;
+			}
+		}
+
+		if (target_buf)
+		{
+			forknum_indexes[count] = i;
+			buf_id_array[count] = curr_buf_id;
+			++count;
+		}
+		else
+		{
+			/*
+			 * It's not the target buffer. Remove the current buffer ID
+			 * from the current list of target buffer IDs and store it
+			 * to a temporary list.
+			 */
+			cb_dlist_delete(hash_entry, curr_buf_id);
+			cb_dlist_push_head(&temp_hash_entry, curr_buf_id);
+		}
+		/* Move current pointer to next */
+		curr_buf_id = new_curr_buf_id;
+	}
+
+	/* Check if main dlist is now empty */
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		hash_entry->head = temp_hash_entry.head;
+		temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+	}
+
+	/* If we have a temporary dlist, append it to the main dlist */
+	if (!cb_dlist_is_empty(hash_entry) &&
+		!cb_dlist_is_empty(&temp_hash_entry))
+		cb_dlist_combine(hash_entry, &temp_hash_entry);
+
+	LWLockRelease(map_lock);
+
+	return count;
+}
+
+/*
+ * CachedBufTableDelete
+ *		Unlink the buffer ID from the doubly-linked list, then remove
+ *		the relation hash entry if its list is empty.
+ *
+ * This function must hold exclusive lock for relation's partition.
+ */
+void
+CachedBufTableDelete(RelFileNode *rnode, int buf_id)
+{
+	LWLock		*map_lock;
+	uint32		hashcode;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	map_lock = GetCachedBufPartitionLock(rnode, &hashcode);
+	LWLockAcquire(map_lock, LW_EXCLUSIVE);
+
+	/* look up hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) rnode,
+									hashcode, HASH_FIND, &found);
+
+	if (!found)		/* rnode not found, nothing to do */
+	{
+		LWLockRelease(map_lock);
+		return;
+	}
+
+	cb_dlist_delete(hash_entry, buf_id);
+
+	/*
+	 * If there's no more cached elements for the target relation,
+	 * remove the relation hash entry.
+	 */
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) rnode,
+										hashcode, HASH_REMOVE, NULL);
+
+		if (!hash_entry)
+			elog(ERROR, "cached buffer hash table corrupted");
+	}
+
+	LWLockRelease(map_lock);
+}
+
+/*
+ * inline functions for the doubly-linked list of cached buffers
+ */
+/* Return the dlist entry */
+static inline BufDlistEnt *
+cb_dlist_entry(int buf_id)
+{
+	return BufDlistEntArray + buf_id;
+}
+
+/* Return the next entry */
+static inline BufDlistEnt *
+cb_dlist_next(BufDlistEnt *entry)
+{
+	return entry->next == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->next);
+}
+
+/* Return the prev entry */
+static inline BufDlistEnt *
+cb_dlist_prev(BufDlistEnt *entry)
+{
+	return entry->prev == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->prev);
+}
+
+/* Return if dlist is empty */
+static inline bool
+cb_dlist_is_empty(CachedBufEnt *hash_entry)
+{
+	return hash_entry->head == CACHED_BUF_END_OF_LIST;
+}
+
+/* Push to head of dlist */
+static inline void
+cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*new_entry = cb_dlist_entry(buf_id);
+	new_entry->next = hash_entry->head;
+
+	if (cb_dlist_is_empty(hash_entry))
+		new_entry->prev = buf_id;
+	else
+	{
+		BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+		new_entry->prev = head_entry->prev;
+		cb_dlist_next(new_entry)->prev = buf_id;
+	}
+	hash_entry->head = buf_id;
+}
+
+/* Remove the buffer ID from dlist */
+static inline void
+cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*curr_entry = cb_dlist_entry(buf_id);
+	BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+	BufDlistEnt	*tail_entry = cb_dlist_entry(head_entry->prev);
+	BufDlistEnt	*next_entry = cb_dlist_next(curr_entry);
+	BufDlistEnt	*prev_entry = cb_dlist_prev(curr_entry);
+
+	/* If entry to be deleted is the only entry */
+	if (head_entry == tail_entry)
+	{
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+		return;
+	}
+
+	/*
+	 * If there is a next entry, update its prev field.
+	 * Otherwise, current entry is at tail (but not at head),
+	 * so update the new tail.
+	 */
+	if (next_entry != NULL)
+		next_entry->prev = curr_entry->prev;
+	else
+		head_entry->prev = curr_entry->prev;
+
+	/*
+	 * If the previous entry is not the tail entry, update its
+	 * next field. Otherwise, current entry is at head (but not
+	 * at tail). Update the new head entry and its tail pointer.
+	 */
+	if (prev_entry != tail_entry)
+		prev_entry->next = curr_entry->next;
+	else
+		hash_entry->head = curr_entry->next;
+}
+
+/* Append head of temporary dlist to main dlist */
+static inline void
+cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp)
+{
+	BufDlistEnt	*main_head_entry = cb_dlist_entry(main->head);
+	BufDlistEnt	*temp_head_entry = cb_dlist_entry(temp->head);
+	int	main_tail = main_head_entry->prev;
+	int	temp_tail = temp_head_entry->prev;
+	BufDlistEnt	*main_tail_entry = cb_dlist_entry(main_tail);
+
+	/* Append the temporary dlist to main dlist */
+	main_tail_entry->next = temp->head;
+	temp_head_entry->prev = main_tail;
+	main_head_entry->prev = temp_tail;
+
+	/* Clear the head of temporary dlist */
+	temp->head = CACHED_BUF_END_OF_LIST;
+}
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index 6ffe184..bb69106 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -338,4 +338,16 @@ extern void DropRelFileNodeLocalBuffers(RelFileNode rnode, ForkNumber forkNum,
 extern void DropRelFileNodeAllLocalBuffers(RelFileNode rnode);
 extern void AtEOXact_LocalBuffers(bool isCommit);
 
+/* in cached_buf.c */
+/* size of array of linked buffer ID's of target relation */
+#define BUF_ID_ARRAY_SIZE	100
+
+extern Size CachedBufShmemSize(void);
+extern void InitCachedBufTable(int size);
+extern void CachedBufTableInsert(RelFileNode *rnode, int buf_id);
+extern int CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				  int *forknum_indexes, BlockNumber *firstDelBlock,
+				  int *buf_id_array, int size);
+extern void CachedBufTableDelete(RelFileNode *rnode, int buf_id);
+
 #endif							/* BUFMGR_INTERNALS_H */
-- 
1.8.3.1

#6Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: k.jamison@fujitsu.com (#5)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Nov 12, 2019 at 10:49:49AM +0000, k.jamison@fujitsu.com wrote:

On Thurs, November 7, 2019 1:27 AM (GMT+9), Robert Haas wrote:

On Tue, Nov 5, 2019 at 10:34 AM Tomas Vondra <tomas.vondra@2ndquadrant.com>
wrote:

2) This adds another hashtable maintenance to BufferAlloc etc. but
you've only done tests / benchmark for the case this optimizes. I
think we need to see a benchmark for workload that allocates and
invalidates lot of buffers. A pgbench with a workload that fits into
RAM but not into shared buffers would be interesting.

Yeah, it seems pretty hard to believe that this won't be bad for some workloads.
Not only do you have the overhead of the hash table operations, but you also
have locking overhead around that. A whole new set of LWLocks where you have
to take and release one of them every time you allocate or invalidate a buffer
seems likely to cause a pretty substantial contention problem.

I'm sorry for the late reply. Thank you Tomas and Robert for checking this patch.
Attached is the v3 of the patch.
- I moved the unnecessary items from buf_internals.h to cached_buf.c since most of
of those items are only used in that file.
- Fixed the bug of v2. Seems to pass both RT and TAP test now

Thanks for the advice on benchmark test. Please refer below for test and results.

[Machine spec]
CPU: 16, Number of cores per socket: 8
RHEL6.5, Memory: 240GB

scale: 3125 (about 46GB DB size)
shared_buffers = 8GB

[workload that fits into RAM but not into shared buffers]
pgbench -i -s 3125 cachetest
pgbench -c 16 -j 8 -T 600 cachetest

[Patched]
scaling factor: 3125
query mode: simple
number of clients: 16
number of threads: 8
duration: 600 s
number of transactions actually processed: 8815123
latency average = 1.089 ms
tps = 14691.436343 (including connections establishing)
tps = 14691.482714 (excluding connections establishing)

[Master/Unpatched]
...
number of transactions actually processed: 8852327
latency average = 1.084 ms
tps = 14753.814648 (including connections establishing)
tps = 14753.861589 (excluding connections establishing)

My patch caused a little overhead of about 0.42-0.46%, which I think is small.
Kindly let me know your opinions/comments about the patch or tests, etc.

Now try measuring that with a read-only workload, with prepared
statements. I've tried that on a machine with 16 cores, doing

# 16 clients
pgbench -n -S -j 16 -c 16 -M prepared -T 60 test

# 1 client
pgbench -n -S -c 1 -M prepared -T 60 test

and average from 30 runs of each looks like this:

# clients master patched %
---------------------------------------------------------
1 29690 27833 93.7%
16 300935 283383 94.1%

That's quite significant regression, considering it's optimizing an
operation that is expected to be pretty rare (people are generally not
dropping dropping objects as often as they query them).

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#7k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Tomas Vondra (#6)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Nov 13, 2019 4:20AM (GMT +9), Tomas Vondra wrote:

On Tue, Nov 12, 2019 at 10:49:49AM +0000, k.jamison@fujitsu.com wrote:

On Thurs, November 7, 2019 1:27 AM (GMT+9), Robert Haas wrote:

On Tue, Nov 5, 2019 at 10:34 AM Tomas Vondra
<tomas.vondra@2ndquadrant.com>
wrote:

2) This adds another hashtable maintenance to BufferAlloc etc. but
you've only done tests / benchmark for the case this optimizes. I
think we need to see a benchmark for workload that allocates and
invalidates lot of buffers. A pgbench with a workload that fits into
RAM but not into shared buffers would be interesting.

Yeah, it seems pretty hard to believe that this won't be bad for some

workloads.

Not only do you have the overhead of the hash table operations, but
you also have locking overhead around that. A whole new set of
LWLocks where you have to take and release one of them every time you
allocate or invalidate a buffer seems likely to cause a pretty substantial

contention problem.

I'm sorry for the late reply. Thank you Tomas and Robert for checking this

patch.

Attached is the v3 of the patch.
- I moved the unnecessary items from buf_internals.h to cached_buf.c
since most of
of those items are only used in that file.
- Fixed the bug of v2. Seems to pass both RT and TAP test now

Thanks for the advice on benchmark test. Please refer below for test and

results.

[Machine spec]
CPU: 16, Number of cores per socket: 8
RHEL6.5, Memory: 240GB

scale: 3125 (about 46GB DB size)
shared_buffers = 8GB

[workload that fits into RAM but not into shared buffers] pgbench -i -s
3125 cachetest pgbench -c 16 -j 8 -T 600 cachetest

[Patched]
scaling factor: 3125
query mode: simple
number of clients: 16
number of threads: 8
duration: 600 s
number of transactions actually processed: 8815123 latency average =
1.089 ms tps = 14691.436343 (including connections establishing) tps =
14691.482714 (excluding connections establishing)

[Master/Unpatched]
...
number of transactions actually processed: 8852327 latency average =
1.084 ms tps = 14753.814648 (including connections establishing) tps =
14753.861589 (excluding connections establishing)

My patch caused a little overhead of about 0.42-0.46%, which I think is small.
Kindly let me know your opinions/comments about the patch or tests, etc.

Now try measuring that with a read-only workload, with prepared statements.
I've tried that on a machine with 16 cores, doing

# 16 clients
pgbench -n -S -j 16 -c 16 -M prepared -T 60 test

# 1 client
pgbench -n -S -c 1 -M prepared -T 60 test

and average from 30 runs of each looks like this:

# clients master patched %
---------------------------------------------------------
1 29690 27833 93.7%
16 300935 283383 94.1%

That's quite significant regression, considering it's optimizing an
operation that is expected to be pretty rare (people are generally not
dropping dropping objects as often as they query them).

I updated the patch and reduced the lock contention of new LWLock,
with tunable definitions in the code and instead of using rnode as the hash key,
I also added the modulo of block number.
#define NUM_MAP_PARTITIONS_FOR_REL 128 /* relation-level */
#define NUM_MAP_PARTITIONS_IN_REL 4 /* block-level */
#define NUM_MAP_PARTITIONS \
(NUM_MAP_PARTITIONS_FOR_REL * NUM_MAP_PARTITIONS_IN_REL)

I executed again a benchmark for read-only workload,
but regression currently sits at 3.10% (reduced from v3's 6%).

Average of 10 runs, 16 clients
read-only, prepared query mode

[Master]
num of txn processed: 11,950,983.67
latency average = 0.080 ms
tps = 199,182.24
tps = 199,189.54

[V4 Patch]
num of txn processed: 11,580,256.36
latency average = 0.083 ms
tps = 193,003.52
tps = 193,010.76

I checked the wait event statistics (non-impactful events omitted)
and got the following below.
I reset the stats before running the pgbench script,
Then showed the stats right after the run.

[Master]
wait_event_type | wait_event | calls | microsec
-----------------+-----------------------+----------+----------
Client | ClientRead | 25116 | 49552452
IO | DataFileRead | 14467109 | 92113056
LWLock | buffer_mapping | 204618 | 1364779

[Patch V4]
wait_event_type | wait_event | calls | microsec
-----------------+-----------------------+----------+----------
Client | ClientRead | 111393 | 68773946
IO | DataFileRead | 14186773 | 90399833
LWLock | buffer_mapping | 463844 | 4025198
LWLock | cached_buf_tranche_id | 83390 | 336080

It seems the buffer_mapping LWLock wait is 4x slower.
However, I'd like to continue working on this patch to next commitfest,
and further reduce its impact to read-only workloads.

Regards,
Kirk Jamison

Attachments:

v4-Optimize-dropping-of-relation-buffers-using-dlist.patchapplication/octet-stream; name=v4-Optimize-dropping-of-relation-buffers-using-dlist.patchDownload
From ed11082ef20329a766080524ff4f5c6b95222c13 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Tue, 12 Nov 2019 08:55:00 +0000
Subject: [PATCH] Optimize dropping of relation buffers using dlist

---
 src/backend/storage/buffer/Makefile     |   1 +
 src/backend/storage/buffer/buf_init.c   |   6 +
 src/backend/storage/buffer/bufmgr.c     |  83 +++---
 src/backend/storage/buffer/cached_buf.c | 486 ++++++++++++++++++++++++++++++++
 src/include/storage/buf_internals.h     |  14 +
 5 files changed, 551 insertions(+), 39 deletions(-)
 create mode 100644 src/backend/storage/buffer/cached_buf.c

diff --git a/src/backend/storage/buffer/Makefile b/src/backend/storage/buffer/Makefile
index fd7c40d..3662349 100644
--- a/src/backend/storage/buffer/Makefile
+++ b/src/backend/storage/buffer/Makefile
@@ -16,6 +16,7 @@ OBJS = \
 	buf_init.o \
 	buf_table.o \
 	bufmgr.o \
+	cached_buf.o \
 	freelist.o \
 	localbuf.o
 
diff --git a/src/backend/storage/buffer/buf_init.c b/src/backend/storage/buffer/buf_init.c
index 1eb08cd..b04e9b2 100644
--- a/src/backend/storage/buffer/buf_init.c
+++ b/src/backend/storage/buffer/buf_init.c
@@ -145,6 +145,9 @@ InitBufferPool(void)
 	/* Init other shared buffer-management stuff */
 	StrategyInitialize(!foundDescs);
 
+	/* Init cached buffer hash table and related shmem data structures */
+	InitCachedBufTable(NBuffers);
+
 	/* Initialize per-backend file flush context */
 	WritebackContextInit(&BackendWritebackContext,
 						 &backend_flush_after);
@@ -188,5 +191,8 @@ BufferShmemSize(void)
 	/* size of checkpoint sort array in bufmgr.c */
 	size = add_size(size, mul_size(NBuffers, sizeof(CkptSortItem)));
 
+	/* size of cached buffer shmem data structures */
+	size = add_size(size, CachedBufShmemSize());
+
 	return size;
 }
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 7ad1073..634273c 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -1268,6 +1268,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 			return buf;
 		}
+		CachedBufTableInsert(&newTag.rnode, &newTag.blockNum, buf->buf_id);
 
 		/*
 		 * Need to lock the buffer header too in order to change its tag.
@@ -1285,6 +1286,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 			break;
 
 		UnlockBufHdr(buf, buf_state);
+		CachedBufTableDelete(&newTag.rnode, &newTag.blockNum, buf->buf_id);
 		BufTableDelete(&newTag, newHash);
 		if (oldPartitionLock != NULL &&
 			oldPartitionLock != newPartitionLock)
@@ -1319,6 +1321,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	if (oldPartitionLock != NULL)
 	{
+		CachedBufTableDelete(&oldTag.rnode, &oldTag.blockNum, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
 		if (oldPartitionLock != newPartitionLock)
 			LWLockRelease(oldPartitionLock);
@@ -1432,7 +1435,10 @@ retry:
 	 * Remove the buffer from the lookup hashtable, if it was in there.
 	 */
 	if (oldFlags & BM_TAG_VALID)
+	{
+		CachedBufTableDelete(&oldTag.rnode, &oldTag.blockNum, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
+	}
 
 	/*
 	 * Done with mapping lock.
@@ -2916,19 +2922,15 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
 DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
-	int			i;
-	int			j;
+	int			i, j, nbufs;
+	int			buf_id_array[BUF_ID_ARRAY_SIZE];
+	int			forknum_indexes[BUF_ID_ARRAY_SIZE];
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2942,45 +2944,48 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	do
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		nbufs = CachedBufLookup(rnode.node, forkNum, nforks,
+								forknum_indexes, firstDelBlock,
+								buf_id_array, lengthof(buf_id_array));
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+		for (i = 0; i < nbufs; i++)
+		{
+			BufferDesc	*bufHdr = GetBufferDescriptor(buf_id_array[i]);
+			uint32		buf_state;
+			int		index = forknum_indexes[i];
 
-		buf_state = LockBufHdr(bufHdr);
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag
+			 * before we attempt to lock the buffer; this saves a lot of
+			 * lock acquisitions in typical cases.  It should be safe
+			 * because the caller must have AccessExclusiveLock on the
+			 * relation, or some other reason to be certain that no one is
+			 * loading new pages of the rel into the buffer pool.
+			 * (Otherwise we might well miss such pages entirely.)
+			 * Therefore, while the tag might be changing while we look at
+			 * it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible,
+			 * and false positives are safe because we'll recheck after
+			 * getting the buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but
+			 * the incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
+
+			buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
-		{
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
-			{
+				bufHdr->tag.forkNum == forkNum[index] &&
+				bufHdr->tag.blockNum >= firstDelBlock[index])
 				InvalidateBuffer(bufHdr); /* releases spinlock */
-				break;
-			}
+			else
+				UnlockBufHdr(bufHdr, buf_state);
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
-	}
+	} while (nbufs == lengthof(buf_id_array));
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/buffer/cached_buf.c b/src/backend/storage/buffer/cached_buf.c
new file mode 100644
index 0000000..9ed0f56
--- /dev/null
+++ b/src/backend/storage/buffer/cached_buf.c
@@ -0,0 +1,486 @@
+/*-------------------------------------------------------------------------
+ *
+ * cached_buf.c
+ *	  routines for mapping relations to the indexes of auxillary cached
+ *	  buffers.
+ *
+ * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ *	  src/backend/storage/buffer/cached_buf.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "storage/bufmgr.h"
+#include "storage/buf_internals.h"
+
+
+/*
+ * Each relation and its buffer information are cached in a hash table
+ * located in shared memory. These cached buffers are chained together
+ * in a doubly-linked list.
+ */
+#define NUM_MAP_PARTITIONS_FOR_REL	128	/* relation-level */
+#define NUM_MAP_PARTITIONS_IN_REL	4	/* block-level */
+#define NUM_MAP_PARTITIONS \
+	(NUM_MAP_PARTITIONS_FOR_REL * NUM_MAP_PARTITIONS_IN_REL)
+#define CACHED_BUF_END_OF_LIST	(-1) /* end of doubly linked list */
+
+/* hash table key */
+typedef struct CachedBufTag
+{
+	RelFileNode		rnode;		/* relation */
+	int			modOfBlockNum;	/* modulo of BlockNumber*/
+} CachedBufTag;
+
+/* entry to the cached buffer hash table */
+typedef struct CachedBufEnt
+{
+	CachedBufTag	key;	/* hash table key */
+	int		head;	/* index of dlist */
+} CachedBufEnt;
+
+/* entry to the doubly-linked list */
+typedef struct BufDlistEnt
+{
+	int		prev;
+	int		next;
+} BufDlistEnt;
+
+/* lock for cached buffer hash table */
+typedef struct CachedBufTableLock
+{
+	LWLock	cacheLock[NUM_MAP_PARTITIONS];
+	int	cacheTrancheId;
+} CachedBufTableLock;
+
+static HTAB *CachedBufHash = NULL;
+static BufDlistEnt *BufDlistEntArray = NULL;
+static CachedBufTableLock *CachedBufLockArray = NULL;
+static uint32 CachedBufTableHashCode(CachedBufTag *tag);
+static LWLock *GetCachedBufPartitionLock(CachedBufTag *tag,
+										 uint32 *hashcode);
+static inline BufDlistEnt *cb_dlist_entry(int buf_id);
+static inline BufDlistEnt *cb_dlist_next(BufDlistEnt *entry);
+static inline BufDlistEnt *cb_dlist_prev(BufDlistEnt *entry);
+static inline bool cb_dlist_is_empty(CachedBufEnt *hash_entry);
+static inline void cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp);
+
+/*
+ * CachedBufShmemSize
+ * 		Estimate space needed for mapping cached buffer hash table
+ *
+ * 		size of lookup table is the desired hash table size
+ *		(possibly more than NBuffers)
+ */
+Size
+CachedBufShmemSize(void)
+{
+	Size		size = 0;
+
+	/* size of cached buffer lookup table */
+	size = add_size(size, hash_estimate_size(NBuffers + NUM_MAP_PARTITIONS,
+											 sizeof(CachedBufEnt)));
+
+	/* size of cached buffer dlist entry array */
+	size = add_size(size, mul_size(NBuffers, sizeof(BufDlistEnt)));
+
+	/* size of locks */
+	size = add_size(size, mul_size(NBuffers,
+								   sizeof(CachedBufTableLock)));
+
+	return size;
+}
+
+/*
+ * InitCachedBufTable
+ *      Initialize the cached buffer hash table and related data
+ *      structures at shared memory initialization.
+ */
+void
+InitCachedBufTable(int size)
+{
+	HASHCTL		info;
+	bool		foundList, foundLock;
+	int		i;
+
+	info.keysize = sizeof(CachedBufTag);
+	info.entrysize = sizeof(CachedBufEnt);
+	info.num_partitions = NUM_MAP_PARTITIONS;
+
+	CachedBufHash = ShmemInitHash("Cached Buffer Lookup Table",
+								  size + NUM_MAP_PARTITIONS,
+								  size + NUM_MAP_PARTITIONS,
+								  &info,
+								  HASH_ELEM | HASH_BLOBS |
+								  HASH_PARTITION);
+
+	BufDlistEntArray = ShmemInitStruct("buffer dlist entry array",
+									   size * sizeof(BufDlistEnt),
+									   &foundList);
+
+	CachedBufLockArray = (CachedBufTableLock *)
+			ShmemInitStruct("cached buffer hash partition lock",
+							size * sizeof(CachedBufTableLock),
+							&foundLock);
+
+	if (!foundList && !foundLock)
+	{
+		CachedBufLockArray->cacheTrancheId = LWLockNewTrancheId();
+
+		for (i = 0; i < NUM_MAP_PARTITIONS; i++)
+			LWLockInitialize(&CachedBufLockArray->cacheLock[i],
+							 CachedBufLockArray->cacheTrancheId);
+	}
+	LWLockRegisterTranche(CachedBufLockArray->cacheTrancheId,
+						  "cached_buf_tranche_id");
+}
+
+ /*
+  * CachedBufTableHashCode
+  *      Compute the hash code associated with tag
+  *
+  * This must be passed to the insert/lookup/delete routines along with the
+  * tag. We do it like this because the callers need to know the hash code
+  * to determine which partition to lock, and we don't want to do the  hash
+  * computation twice (hash_any is a bit slow).
+  */
+static uint32
+CachedBufTableHashCode(CachedBufTag *tag)
+{
+	return get_hash_value(CachedBufHash, (void *) tag);
+}
+
+/*
+ * GetCachedBufPartitionLock
+ * 		Get lock instance for partition of cached buffer lookup table
+ */
+static LWLock *
+GetCachedBufPartitionLock(CachedBufTag *tag, uint32 *hashcode)
+{
+	*hashcode = CachedBufTableHashCode(tag);
+
+	return &CachedBufLockArray->cacheLock[*hashcode % NUM_MAP_PARTITIONS];
+}
+
+/*
+ * CachedBufTableInsert
+ *		Insert a hash table entry for given tag and buffer ID
+ *
+ * If a hash entry for the tag already exists, only buffer ID is inserted.
+ * Chain the buffer ID to the doubly-linked list.
+ *
+ * This function must hold exclusive lock for tag's partition
+ */
+void
+CachedBufTableInsert(RelFileNode *rnode, BlockNumber *blockNum, int buf_id)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	LWLock		*map_lock;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	cb_tag.rnode = *rnode;
+	cb_tag.modOfBlockNum = *blockNum % NUM_MAP_PARTITIONS_IN_REL;
+
+	map_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+	LWLockAcquire(map_lock, LW_EXCLUSIVE);
+
+	/* look up or create a hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &cb_tag,
+									hashcode,
+									HASH_ENTER,
+									&found);
+
+	/* If not found, initialize linked list */
+	if (!found)
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+
+	cb_dlist_push_head(hash_entry, buf_id);
+
+	LWLockRelease(map_lock);
+}
+
+/*
+ * CachedBufLookup
+ *		Lookup the buffers for the given tag in the cached buffer hash
+ *		table. Insert the buffer ID to the array of buffer IDs, and
+ *		return the total number of buffers in the array.
+ *
+ * This function must hold exclusive LWLock for tag's partition
+ */
+int
+CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				int *forknum_indexes, BlockNumber *firstDelBlock,
+				int *buf_id_array, int size)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	LWLock		*map_lock;
+	CachedBufEnt	*hash_entry;
+	CachedBufEnt	temp_hash_entry;
+	BufDlistEnt	*curr_entry = NULL;
+	int		mod;
+	int		i;
+	int		curr_buf_id;
+	int		new_curr_buf_id;
+	int		count = 0;
+	bool		target_buf = false;
+
+	for (mod = 0; mod < NUM_MAP_PARTITIONS_IN_REL; mod++)
+	{
+		cb_tag.rnode = rnode;
+		cb_tag.modOfBlockNum = mod;
+
+		map_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+		LWLockAcquire(map_lock, LW_EXCLUSIVE);
+
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) &cb_tag,
+										hashcode,
+										HASH_FIND,
+										NULL);
+
+		if (!hash_entry)
+		{
+			LWLockRelease(map_lock);
+			continue;
+		}
+
+		/* Initial temporary dlist */
+		temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+
+		/* When traversing the main dlist, start from head */
+		curr_buf_id = hash_entry->head;
+
+		while(curr_buf_id != CACHED_BUF_END_OF_LIST && count < size)
+		{
+			BufferDesc *bufHdr = GetBufferDescriptor(curr_buf_id);
+			curr_entry = BufDlistEntArray + curr_buf_id;
+			new_curr_buf_id = curr_entry->next;
+
+			/* Check if it's our target buffer */
+			for (i = 0; i < nforks; i++)
+			{
+				if (bufHdr->tag.forkNum != forkNum[i])
+					continue;
+				else
+				{
+					if (bufHdr->tag.blockNum >= firstDelBlock[i])
+						target_buf = true;
+					break;
+				}
+			}
+
+			if (target_buf)
+			{
+				forknum_indexes[count] = i;
+				buf_id_array[count] = curr_buf_id;
+				++count;
+			}
+			else
+			{
+				/*
+				* It's not the target buffer. Remove the current buffer ID
+				* from the current list of target buffer IDs and store it
+				* to a temporary list.
+				*/
+				cb_dlist_delete(hash_entry, curr_buf_id);
+				cb_dlist_push_head(&temp_hash_entry, curr_buf_id);
+			}
+			/* Move current pointer to next */
+			curr_buf_id = new_curr_buf_id;
+		}
+
+		/* Check if main dlist is now empty */
+		if (cb_dlist_is_empty(hash_entry))
+		{
+			hash_entry->head = temp_hash_entry.head;
+			temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+		}
+
+		/* If we have a temporary dlist, append it to the main dlist */
+		if (!cb_dlist_is_empty(hash_entry) &&
+			!cb_dlist_is_empty(&temp_hash_entry))
+			cb_dlist_combine(hash_entry, &temp_hash_entry);
+
+		LWLockRelease(map_lock);
+	}
+
+	return count;
+}
+
+/*
+ * CachedBufTableDelete
+ *		Unlink the buffer ID from the doubly-linked list, then remove
+ *		the hash entry for the given tag if its list is empty.
+ *
+ * This function must hold exclusive lock for tag's partition
+ */
+void
+CachedBufTableDelete(RelFileNode *rnode, BlockNumber *blockNum, int buf_id)
+{
+	CachedBufTag	cb_tag;
+	LWLock		*map_lock;
+	uint32		hashcode;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	cb_tag.rnode = *rnode;
+	cb_tag.modOfBlockNum = *blockNum % NUM_MAP_PARTITIONS_IN_REL;
+
+	map_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+	LWLockAcquire(map_lock, LW_EXCLUSIVE);
+
+	/* look up hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &cb_tag,
+									hashcode,
+									HASH_FIND,
+									&found);
+
+	if (!found)		/* tag not found, nothing to do */
+	{
+		LWLockRelease(map_lock);
+		return;
+	}
+
+	cb_dlist_delete(hash_entry, buf_id);
+
+	/*
+	 * If there's no more cached elements for the given tag,
+	 * remove the hash entry.
+	 */
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) &cb_tag,
+										hashcode,
+										HASH_REMOVE,
+										NULL);
+
+		if (!hash_entry)
+			elog(ERROR, "cached buffer hash table corrupted");
+	}
+
+	LWLockRelease(map_lock);
+}
+
+/*
+ * inline functions for the doubly-linked list of cached buffers
+ */
+/* Return the dlist entry */
+static inline BufDlistEnt *
+cb_dlist_entry(int buf_id)
+{
+	return BufDlistEntArray + buf_id;
+}
+
+/* Return the next entry */
+static inline BufDlistEnt *
+cb_dlist_next(BufDlistEnt *entry)
+{
+	return entry->next == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->next);
+}
+
+/* Return the prev entry */
+static inline BufDlistEnt *
+cb_dlist_prev(BufDlistEnt *entry)
+{
+	return entry->prev == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->prev);
+}
+
+/* Return if dlist is empty */
+static inline bool
+cb_dlist_is_empty(CachedBufEnt *hash_entry)
+{
+	return hash_entry->head == CACHED_BUF_END_OF_LIST;
+}
+
+/* Push to head of dlist */
+static inline void
+cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*new_entry = cb_dlist_entry(buf_id);
+	new_entry->next = hash_entry->head;
+
+	if (cb_dlist_is_empty(hash_entry))
+		new_entry->prev = buf_id;
+	else
+	{
+		BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+		new_entry->prev = head_entry->prev;
+		cb_dlist_next(new_entry)->prev = buf_id;
+	}
+	hash_entry->head = buf_id;
+}
+
+/* Remove the buffer ID from dlist */
+static inline void
+cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*curr_entry = cb_dlist_entry(buf_id);
+	BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+	BufDlistEnt	*tail_entry = cb_dlist_entry(head_entry->prev);
+	BufDlistEnt	*next_entry = cb_dlist_next(curr_entry);
+	BufDlistEnt	*prev_entry = cb_dlist_prev(curr_entry);
+
+	/* If only one entry is in the list */
+	if (head_entry == tail_entry)
+	{
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+		return;
+	}
+
+	/*
+	 * If there is a next entry, update its prev field.
+	 * Otherwise, current entry is at tail (but not at head),
+	 * so update the new tail.
+	 */
+	if (next_entry != NULL)
+		next_entry->prev = curr_entry->prev;
+	else
+		head_entry->prev = curr_entry->prev;
+
+	/*
+	 * If the previous entry is not the tail entry, update its
+	 * next field. Otherwise, current entry is at head (but not
+	 * at tail). Update the new head entry and its tail pointer.
+	 */
+	if (prev_entry != tail_entry)
+		prev_entry->next = curr_entry->next;
+	else
+		hash_entry->head = curr_entry->next;
+}
+
+/* Append head of temporary dlist to main dlist */
+static inline void
+cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp)
+{
+	BufDlistEnt	*main_head_entry = cb_dlist_entry(main->head);
+	BufDlistEnt	*temp_head_entry = cb_dlist_entry(temp->head);
+	int	main_tail = main_head_entry->prev;
+	int	temp_tail = temp_head_entry->prev;
+	BufDlistEnt	*main_tail_entry = cb_dlist_entry(main_tail);
+
+	/* Append the temporary dlist to main dlist */
+	main_tail_entry->next = temp->head;
+	temp_head_entry->prev = main_tail;
+	main_head_entry->prev = temp_tail;
+
+	/* Clear the head of temporary dlist */
+	temp->head = CACHED_BUF_END_OF_LIST;
+}
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index d540c0a..1068815 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -337,4 +337,18 @@ extern void DropRelFileNodeLocalBuffers(RelFileNode rnode, ForkNumber forkNum,
 extern void DropRelFileNodeAllLocalBuffers(RelFileNode rnode);
 extern void AtEOXact_LocalBuffers(bool isCommit);
 
+/* in cached_buf.c */
+/* size of array of linked buffer ID's of target relation */
+#define BUF_ID_ARRAY_SIZE	100
+
+extern Size CachedBufShmemSize(void);
+extern void InitCachedBufTable(int size);
+extern void CachedBufTableInsert(RelFileNode *rnode, BlockNumber *blockNum,
+								 int buf_id);
+extern int CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				  int *forknum_indexes, BlockNumber *firstDelBlock,
+				  int *buf_id_array, int size);
+extern void CachedBufTableDelete(RelFileNode *rnode,  BlockNumber *blockNum,
+								 int buf_id);
+
 #endif							/* BUFMGR_INTERNALS_H */
-- 
1.8.3.1

#8k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#7)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi,

I have updated the patch (v5).
I tried to reduce the lock waiting times by using spinlock
when inserting/deleting buffers in the new hash table, and
exclusive lock when doing lookup for buffers to be dropped.
In summary, instead of scanning the whole buffer pool in
shared buffers, we just traverse the doubly-linked list of linked
buffers for the target relation and block.

In order to understand how this patch affects performance,
I also measured the cache hit rates in addition to
benchmarking db with various shared buffer size settings.

Using the same machine specs, I used the default script
of pgbench for read-only workload with prepared statement,
and executed about 15 runs for varying shared buffer sizes.
pgbench -i -s 3200 test //(about 48GB db size)
pgbench -S -n -M prepared -c 16 -j 16 -T 60 test

[TPS Regression]
shbuf | tps(master) | tps(patch) | %reg
---------+-----------------+-----------------+-------
5GB | 195,737.23 | 191,422.23 | 2.23
10GB | 197,067.93 | 194,011.66 | 1.55
20GB | 200,241.18 | 200,425.29 | -0.09
40GB | 208,772.81 | 209,807.38 | -0.50
50GB | 215,684.33 | 218,955.43 | -1.52

[CACHE HIT RATE]
Shbuf | master | patch
----------+--------------+----------
10GB | 0.141536 | 0.141485
20GB | 0.330088 | 0.329894
30GB | 0.573383 | 0.573377
40GB | 0.819499 | 0.819264
50GB | 0.999237 | 0.999577

For this workload, the regression increases for below 20GB
shared_buffers size. However, the cache hit rate both for
master and patch is 32% (20 GB shbuf). Therefore, I think we
can consider this kind of workload with low shared buffers
size as a “special case”, because in terms of db performance
tuning we want as much as possible for the db to have a higher
cache hit rate (99.9%, or maybe let's say 80% is acceptable).
And in this workload, ideal shared_buffers size would be
around 40GB more or less to hit that acceptable cache hit rate.
Looking at this patch's performance result, if it's within the acceptable
cache hit rate, there would be at least no regression and the results als
show almost similar tps compared to master.

Your feedback about the patch and tests are welcome.

Regards,
Kirk Jamison

Attachments:

v5-Optimize-dropping-of-relation-buffers-using-dlist.patchapplication/octet-stream; name=v5-Optimize-dropping-of-relation-buffers-using-dlist.patchDownload
From d3ef35f28494255d6a7f1d7a80d088f491f29ffd Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Tue, 12 Nov 2019 08:55:00 +0000
Subject: [PATCH] Optimize dropping of relation buffers using dlist

---
 src/backend/storage/buffer/Makefile     |   1 +
 src/backend/storage/buffer/buf_init.c   |   6 +
 src/backend/storage/buffer/bufmgr.c     |  88 ++---
 src/backend/storage/buffer/cached_buf.c | 571 ++++++++++++++++++++++++++++++++
 src/include/storage/buf_internals.h     |  14 +
 5 files changed, 639 insertions(+), 41 deletions(-)
 create mode 100644 src/backend/storage/buffer/cached_buf.c

diff --git a/src/backend/storage/buffer/Makefile b/src/backend/storage/buffer/Makefile
index fd7c40d..3662349 100644
--- a/src/backend/storage/buffer/Makefile
+++ b/src/backend/storage/buffer/Makefile
@@ -16,6 +16,7 @@ OBJS = \
 	buf_init.o \
 	buf_table.o \
 	bufmgr.o \
+	cached_buf.o \
 	freelist.o \
 	localbuf.o
 
diff --git a/src/backend/storage/buffer/buf_init.c b/src/backend/storage/buffer/buf_init.c
index 1eb08cd..b04e9b2 100644
--- a/src/backend/storage/buffer/buf_init.c
+++ b/src/backend/storage/buffer/buf_init.c
@@ -145,6 +145,9 @@ InitBufferPool(void)
 	/* Init other shared buffer-management stuff */
 	StrategyInitialize(!foundDescs);
 
+	/* Init cached buffer hash table and related shmem data structures */
+	InitCachedBufTable(NBuffers);
+
 	/* Initialize per-backend file flush context */
 	WritebackContextInit(&BackendWritebackContext,
 						 &backend_flush_after);
@@ -188,5 +191,8 @@ BufferShmemSize(void)
 	/* size of checkpoint sort array in bufmgr.c */
 	size = add_size(size, mul_size(NBuffers, sizeof(CkptSortItem)));
 
+	/* size of cached buffer shmem data structures */
+	size = add_size(size, CachedBufShmemSize());
+
 	return size;
 }
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 7ad1073..4e6e52f 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -1268,6 +1268,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 			return buf;
 		}
+		CachedBufTableInsert(&newTag.rnode, &newTag.blockNum, buf->buf_id);
 
 		/*
 		 * Need to lock the buffer header too in order to change its tag.
@@ -1285,6 +1286,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 			break;
 
 		UnlockBufHdr(buf, buf_state);
+		CachedBufTableDelete(&newTag.rnode, &newTag.blockNum, buf->buf_id);
 		BufTableDelete(&newTag, newHash);
 		if (oldPartitionLock != NULL &&
 			oldPartitionLock != newPartitionLock)
@@ -1319,6 +1321,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	if (oldPartitionLock != NULL)
 	{
+		CachedBufTableDelete(&oldTag.rnode, &oldTag.blockNum, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
 		if (oldPartitionLock != newPartitionLock)
 			LWLockRelease(oldPartitionLock);
@@ -1432,7 +1435,10 @@ retry:
 	 * Remove the buffer from the lookup hashtable, if it was in there.
 	 */
 	if (oldFlags & BM_TAG_VALID)
+	{
+		CachedBufTableDelete(&oldTag.rnode, &oldTag.blockNum, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
+	}
 
 	/*
 	 * Done with mapping lock.
@@ -2916,11 +2922,6 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
@@ -2928,59 +2929,64 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
-	int			j;
+	int			nbufs;
+	int			buf_id_array[BUF_ID_ARRAY_SIZE];
+	int			forknum_indexes[BUF_ID_ARRAY_SIZE];
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
 	{
 		if (rnode.backend == MyBackendId)
 		{
-			for (j = 0; j < nforks; j++)
-				DropRelFileNodeLocalBuffers(rnode.node, forkNum[j],
-											firstDelBlock[j]);
+			for (i = 0; i < nforks; i++)
+				DropRelFileNodeLocalBuffers(rnode.node, forkNum[i],
+											firstDelBlock[i]);
 		}
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	do
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		nbufs = CachedBufLookup(rnode.node, forkNum, nforks,
+								forknum_indexes, firstDelBlock,
+								buf_id_array, lengthof(buf_id_array));
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+		for (i = 0; i < nbufs; i++)
+		{
+			BufferDesc	*bufHdr = GetBufferDescriptor(buf_id_array[i]);
+			uint32		buf_state;
+			int		index = forknum_indexes[i];
 
-		buf_state = LockBufHdr(bufHdr);
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag
+			 * before we attempt to lock the buffer; this saves a lot of
+			 * lock acquisitions in typical cases.  It should be safe
+			 * because the caller must have AccessExclusiveLock on the
+			 * relation, or some other reason to be certain that no one is
+			 * loading new pages of the rel into the buffer pool.
+			 * (Otherwise we might well miss such pages entirely.)
+			 * Therefore, while the tag might be changing while we look at
+			 * it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible,
+			 * and false positives are safe because we'll recheck after
+			 * getting the buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but
+			 * the incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
+
+			buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
-		{
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
-			{
+				bufHdr->tag.forkNum == forkNum[index] &&
+				bufHdr->tag.blockNum >= firstDelBlock[index])
 				InvalidateBuffer(bufHdr); /* releases spinlock */
-				break;
-			}
+			else
+				UnlockBufHdr(bufHdr, buf_state);
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
-	}
+	} while (nbufs == lengthof(buf_id_array));
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/buffer/cached_buf.c b/src/backend/storage/buffer/cached_buf.c
new file mode 100644
index 0000000..066b143
--- /dev/null
+++ b/src/backend/storage/buffer/cached_buf.c
@@ -0,0 +1,571 @@
+/*-------------------------------------------------------------------------
+ *
+ * cached_buf.c
+ *	  routines for mapping relations to the indexes of auxillary cached
+ *	  buffers.
+ *
+ * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ *	  src/backend/storage/buffer/cached_buf.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "storage/bufmgr.h"
+#include "storage/buf_internals.h"
+
+
+/*
+ * Each relation and its buffer information are cached in a hash table
+ * located in shared memory. These cached buffers are chained together
+ * in a doubly-linked list.
+ */
+#define NUM_MAP_PARTITIONS_FOR_REL	128	/* relation-level */
+#define NUM_MAP_PARTITIONS_IN_REL	4	/* block-level */
+#define NUM_MAP_PARTITIONS \
+	(NUM_MAP_PARTITIONS_FOR_REL * NUM_MAP_PARTITIONS_IN_REL)
+#define CACHED_BUF_END_OF_LIST	(-1) /* end of doubly linked list */
+
+/* hash table key */
+typedef struct CachedBufTag
+{
+	RelFileNode		rnode;		/* relation */
+	int			modOfBlockNum;	/* modulo of BlockNumber*/
+} CachedBufTag;
+
+/* entry to the cached buffer hash table */
+typedef struct CachedBufEnt
+{
+	CachedBufTag	key;	/* hash table key */
+	int		head;	/* index of dlist */
+} CachedBufEnt;
+
+/* entry to the doubly-linked list */
+typedef struct BufDlistEnt
+{
+	int		prev;
+	int		next;
+} BufDlistEnt;
+
+/* Locks to be used for the hash table operations */
+typedef struct CombinedLock
+{
+	volatile bool	flag;	/* flag for lookup operation */
+	slock_t		spinLock;	/* spinlock to protect the flag */
+	LWLock		lwLock;	/* LWLock used in lookup */
+} CombinedLock;
+
+/* lock for cached buffer hash table */
+typedef struct CachedBufTableLock
+{
+	CombinedLock	cacheLock[NUM_MAP_PARTITIONS];
+	int		cacheTrancheId;
+} CachedBufTableLock;
+
+static HTAB *CachedBufHash = NULL;
+static BufDlistEnt *BufDlistEntArray = NULL;
+static CachedBufTableLock *CachedBufLockArray = NULL;
+static void CombinedLockInitialize(CombinedLock *lock, int trancheId);
+static uint32 CachedBufTableHashCode(CachedBufTag *tag);
+static CombinedLock *GetCachedBufPartitionLock(CachedBufTag *tag,
+											   uint32 *hashcode);
+static void CombinedLockAcquireSpinLock(CombinedLock *lock);
+static void CombinedLockAcquireLWLock(CombinedLock *lock);
+static void CombinedLockReleaseSpinLock(CombinedLock *lock);
+static void CombinedLockReleaseLWLock(CombinedLock *lock);
+static inline BufDlistEnt *cb_dlist_entry(int buf_id);
+static inline BufDlistEnt *cb_dlist_next(BufDlistEnt *entry);
+static inline BufDlistEnt *cb_dlist_prev(BufDlistEnt *entry);
+static inline bool cb_dlist_is_empty(CachedBufEnt *hash_entry);
+static inline void cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp);
+
+/*
+ * CachedBufShmemSize
+ * 		Estimate space needed for mapping cached buffer hash table
+ *
+ * 		size of lookup table is the desired hash table size
+ *		(possibly more than NBuffers)
+ */
+Size
+CachedBufShmemSize(void)
+{
+	Size		size = 0;
+
+	/* size of cached buffer lookup table */
+	size = add_size(size, hash_estimate_size(NBuffers +
+											 NUM_MAP_PARTITIONS,
+											 sizeof(CachedBufEnt)));
+
+	/* size of cached buffer dlist entry array */
+	size = add_size(size, mul_size(NBuffers, sizeof(BufDlistEnt)));
+
+	/* size of locks */
+	size = add_size(size, mul_size(NBuffers,
+								   sizeof(CachedBufTableLock)));
+
+	return size;
+}
+
+/* Initialize spinlock and LWLock for cached buffer hash table */
+void
+CombinedLockInitialize(CombinedLock *lock, int trancheId)
+{
+	lock->flag = false;
+	SpinLockInit(&lock->spinLock);
+	LWLockInitialize(&lock->lwLock, trancheId);
+}
+
+/*
+ * InitCachedBufTable
+ *      Initialize the cached buffer hash table and related data
+ *      structures at shared memory initialization.
+ */
+void
+InitCachedBufTable(int size)
+{
+	HASHCTL		info;
+	bool		foundList;
+	bool		foundLock;
+	int		i;
+
+	info.keysize = sizeof(CachedBufTag);
+	info.entrysize = sizeof(CachedBufEnt);
+	info.num_partitions = NUM_MAP_PARTITIONS;
+
+	CachedBufHash = ShmemInitHash("Cached Buffer Lookup Table",
+								  size + NUM_MAP_PARTITIONS,
+								  size + NUM_MAP_PARTITIONS,
+								  &info,
+								  HASH_ELEM | HASH_BLOBS |
+								  HASH_PARTITION);
+
+	BufDlistEntArray = ShmemInitStruct("buffer dlist entry array",
+									   size * sizeof(BufDlistEnt),
+									   &foundList);
+
+	CachedBufLockArray = (CachedBufTableLock *)
+			ShmemInitStruct("cached buffer hash partition lock",
+							size * sizeof(CachedBufTableLock),
+							&foundLock);
+
+	if (!foundList && !foundLock)
+	{
+		CachedBufLockArray->cacheTrancheId = LWLockNewTrancheId();
+
+		for (i = 0; i < NUM_MAP_PARTITIONS; i++)
+			CombinedLockInitialize(&CachedBufLockArray->cacheLock[i],
+								   CachedBufLockArray->cacheTrancheId);
+	}
+	LWLockRegisterTranche(CachedBufLockArray->cacheTrancheId,
+						  "cached_buf_tranche_id");
+
+}
+
+ /*
+  * CachedBufTableHashCode
+  *      Compute the hash code associated with tag
+  *
+  * This must be passed to the insert/lookup/delete routines along with the
+  * tag. We do it like this because the callers need to know the hash code
+  * to determine which partition to lock, and we don't want to do the  hash
+  * computation twice (hash_any is a bit slow).
+  */
+static uint32
+CachedBufTableHashCode(CachedBufTag *tag)
+{
+	return get_hash_value(CachedBufHash, (void *) tag);
+}
+
+/*
+ * GetCachedBufPartitionLock
+ * 		Get lock instance for partition of cached buffer lookup table
+ */
+static CombinedLock *
+GetCachedBufPartitionLock(CachedBufTag *tag, uint32 *hashcode)
+{
+	*hashcode = CachedBufTableHashCode(tag);
+
+	return &CachedBufLockArray->cacheLock[*hashcode % NUM_MAP_PARTITIONS];
+}
+
+/*
+ * CombinedLockAcquireSpinLock
+ *		Acquire spinlock to proceed to inserting/deleting hash
+ *		table entries
+ *
+ * This function waits for a lookup process (if flag is true) to finish,
+ * before reacquring a spinlock.
+ */
+void
+CombinedLockAcquireSpinLock(CombinedLock *lock)
+{
+	SpinLockAcquire(&lock->spinLock);
+
+	while (lock->flag)
+	{
+		SpinLockRelease(&lock->spinLock);
+		LWLockAcquire(&lock->lwLock, LW_EXCLUSIVE);
+		LWLockRelease(&lock->lwLock);
+		SpinLockAcquire(&lock->spinLock);
+	}
+}
+
+/*
+ * CombinedLockAcquireLWLock
+ *		Get LWLock instance for partition of cached buffer lookup table
+ *
+ * To prevent other backend processes from modifying the lookup table,
+ * this function sets the CombinedLock's flag to true.
+ */
+void
+CombinedLockAcquireLWLock(CombinedLock *lock)
+{
+	LWLockAcquire(&lock->lwLock, LW_EXCLUSIVE);
+	SpinLockAcquire(&lock->spinLock);
+	lock->flag = true;
+	SpinLockRelease(&lock->spinLock);
+}
+
+/* 
+ * CombinedLockReleaseSpinLock
+ *		Release the spinlock used in insert/delete functions 
+ */
+void
+CombinedLockReleaseSpinLock(CombinedLock *lock)
+{
+	SpinLockRelease(&lock->spinLock);
+}
+
+/* 
+ * CombinedLockReleaseSpinLock
+ *		Release the LWLock used in lookup function after changing the
+ *		CombinedLock's flag to false.
+ */
+void
+CombinedLockReleaseLWLock(CombinedLock *lock)
+{
+	lock->flag = false;
+	LWLockRelease(&lock->lwLock);
+}
+
+/*
+ * CachedBufTableInsert
+ *		Insert a hash table entry for given tag and buffer ID
+ *
+ * If a hash entry for the tag already exists, only buffer ID is inserted.
+ * Chain the buffer ID to the doubly-linked list.
+ *
+ * This function must hold spin lock.
+ */
+void
+CachedBufTableInsert(RelFileNode *rnode, BlockNumber *blockNum, int buf_id)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	cb_tag.rnode = *rnode;
+	cb_tag.modOfBlockNum = *blockNum % NUM_MAP_PARTITIONS_IN_REL;
+
+	combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+	CombinedLockAcquireSpinLock(combined_lock);
+
+	/* look up or create a hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &cb_tag,
+									hashcode,
+									HASH_ENTER,
+									&found);
+
+	/* If not found, initialize linked list */
+	if (!found)
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+
+	cb_dlist_push_head(hash_entry, buf_id);
+
+	CombinedLockReleaseSpinLock(combined_lock);
+}
+
+/*
+ * CachedBufLookup
+ *		Lookup the buffers for the given tag in the cached buffer
+ *		hash table. Insert the buffer ID to the array of buffer
+ *		IDs, and return the total number of buffers in the array.
+ *
+ * This function must hold exclusive LWLock for tag's partition.
+ */
+int
+CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				int *forknum_indexes, BlockNumber *firstDelBlock,
+				int *buf_id_array, int size)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	CachedBufEnt	temp_hash_entry;
+	BufDlistEnt	*curr_entry = NULL;
+	int		mod;
+	int		i;
+	int		curr_buf_id;
+	int		new_curr_buf_id;
+	int		count = 0;
+	bool		target_buf = false;
+
+	for (mod = 0; mod < NUM_MAP_PARTITIONS_IN_REL; mod++)
+	{
+		cb_tag.rnode = rnode;
+		cb_tag.modOfBlockNum = mod;
+
+		combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+		CombinedLockAcquireLWLock(combined_lock);
+
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) &cb_tag,
+										hashcode,
+										HASH_FIND,
+										NULL);
+
+		if (!hash_entry)
+		{
+			CombinedLockReleaseLWLock(combined_lock);
+			continue;
+		}
+
+		/* Initial temporary dlist */
+		temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+
+		/* When traversing the main dlist, start from head */
+		curr_buf_id = hash_entry->head;
+
+		while(curr_buf_id != CACHED_BUF_END_OF_LIST && count < size)
+		{
+			BufferDesc *bufHdr = GetBufferDescriptor(curr_buf_id);
+			curr_entry = BufDlistEntArray + curr_buf_id;
+			new_curr_buf_id = curr_entry->next;
+
+			/* Check if it's our target buffer */
+			for (i = 0; i < nforks; i++)
+			{
+				if (bufHdr->tag.forkNum != forkNum[i])
+					continue;
+				else
+				{
+					if (bufHdr->tag.blockNum >= firstDelBlock[i])
+						target_buf = true;
+					break;
+				}
+			}
+
+			if (target_buf)
+			{
+				forknum_indexes[count] = i;
+				buf_id_array[count] = curr_buf_id;
+				++count;
+			}
+			else
+			{
+				/*
+				* It's not the target buffer. Remove the current buffer ID
+				* from the current list of target buffer IDs and store it
+				* to a temporary list.
+				*/
+				cb_dlist_delete(hash_entry, curr_buf_id);
+				cb_dlist_push_head(&temp_hash_entry, curr_buf_id);
+			}
+			/* Move current pointer to next */
+			curr_buf_id = new_curr_buf_id;
+		}
+
+		/* Check if main dlist is now empty */
+		if (cb_dlist_is_empty(hash_entry))
+		{
+			hash_entry->head = temp_hash_entry.head;
+			temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+		}
+
+		/* If we have a temporary dlist, append it to the main dlist */
+		if (!cb_dlist_is_empty(hash_entry) &&
+			!cb_dlist_is_empty(&temp_hash_entry))
+			cb_dlist_combine(hash_entry, &temp_hash_entry);
+
+		CombinedLockReleaseLWLock(combined_lock);
+	}
+
+	return count;
+}
+
+/*
+ * CachedBufTableDelete
+ *		Unlink the buffer ID from the doubly-linked list, then remove
+ *		the hash entry for the given tag if its list is empty.
+ *
+ * This function must hold spin lock.
+ */
+void
+CachedBufTableDelete(RelFileNode *rnode, BlockNumber *blockNum, int buf_id)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	cb_tag.rnode = *rnode;
+	cb_tag.modOfBlockNum = *blockNum % NUM_MAP_PARTITIONS_IN_REL;
+
+	combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+	CombinedLockAcquireSpinLock(combined_lock);
+
+	/* look up hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &cb_tag,
+									hashcode,
+									HASH_FIND,
+									&found);
+
+	if (!found)		/* tag not found, nothing to do */
+	{
+		CombinedLockReleaseSpinLock(combined_lock);
+		return;
+	}
+
+	cb_dlist_delete(hash_entry, buf_id);
+
+	/*
+	 * If there's no more cached elements for the given tag,
+	 * remove the hash entry.
+	 */
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) &cb_tag,
+										hashcode,
+										HASH_REMOVE,
+										NULL);
+
+		if (!hash_entry)
+			elog(ERROR, "cached buffer hash table corrupted");
+	}
+
+	CombinedLockReleaseSpinLock(combined_lock);
+}
+
+/*
+ * inline functions for the doubly-linked list of cached buffers
+ */
+/* Return the dlist entry */
+static inline BufDlistEnt *
+cb_dlist_entry(int buf_id)
+{
+	return BufDlistEntArray + buf_id;
+}
+
+/* Return the next entry */
+static inline BufDlistEnt *
+cb_dlist_next(BufDlistEnt *entry)
+{
+	return entry->next == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->next);
+}
+
+/* Return the prev entry */
+static inline BufDlistEnt *
+cb_dlist_prev(BufDlistEnt *entry)
+{
+	return entry->prev == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->prev);
+}
+
+/* Return if dlist is empty */
+static inline bool
+cb_dlist_is_empty(CachedBufEnt *hash_entry)
+{
+	return hash_entry->head == CACHED_BUF_END_OF_LIST;
+}
+
+/* Push to head of dlist */
+static inline void
+cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*new_entry = cb_dlist_entry(buf_id);
+	new_entry->next = hash_entry->head;
+
+	if (cb_dlist_is_empty(hash_entry))
+		new_entry->prev = buf_id;
+	else
+	{
+		BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+		new_entry->prev = head_entry->prev;
+		cb_dlist_next(new_entry)->prev = buf_id;
+	}
+	hash_entry->head = buf_id;
+}
+
+/* Remove the buffer ID from dlist */
+static inline void
+cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*curr_entry = cb_dlist_entry(buf_id);
+	BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+	BufDlistEnt	*tail_entry = cb_dlist_entry(head_entry->prev);
+	BufDlistEnt	*next_entry = cb_dlist_next(curr_entry);
+	BufDlistEnt	*prev_entry = cb_dlist_prev(curr_entry);
+
+	/* If only one entry is in the list */
+	if (head_entry == tail_entry)
+	{
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+		return;
+	}
+
+	/*
+	 * If there is a next entry, update its prev field.
+	 * Otherwise, current entry is at tail (but not at head),
+	 * so update the new tail.
+	 */
+	if (next_entry != NULL)
+		next_entry->prev = curr_entry->prev;
+	else
+		head_entry->prev = curr_entry->prev;
+
+	/*
+	 * If the previous entry is not the tail entry, update its
+	 * next field. Otherwise, current entry is at head (but not
+	 * at tail). Update the new head entry and its tail pointer.
+	 */
+	if (prev_entry != tail_entry)
+		prev_entry->next = curr_entry->next;
+	else
+		hash_entry->head = curr_entry->next;
+}
+
+/* Append head of temporary dlist to main dlist */
+static inline void
+cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp)
+{
+	BufDlistEnt	*main_head_entry = cb_dlist_entry(main->head);
+	BufDlistEnt	*temp_head_entry = cb_dlist_entry(temp->head);
+	int	main_tail = main_head_entry->prev;
+	int	temp_tail = temp_head_entry->prev;
+	BufDlistEnt	*main_tail_entry = cb_dlist_entry(main_tail);
+
+	/* Append the temporary dlist to main dlist */
+	main_tail_entry->next = temp->head;
+	temp_head_entry->prev = main_tail;
+	main_head_entry->prev = temp_tail;
+
+	/* Clear the head of temporary dlist */
+	temp->head = CACHED_BUF_END_OF_LIST;
+}
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index d540c0a..1068815 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -337,4 +337,18 @@ extern void DropRelFileNodeLocalBuffers(RelFileNode rnode, ForkNumber forkNum,
 extern void DropRelFileNodeAllLocalBuffers(RelFileNode rnode);
 extern void AtEOXact_LocalBuffers(bool isCommit);
 
+/* in cached_buf.c */
+/* size of array of linked buffer ID's of target relation */
+#define BUF_ID_ARRAY_SIZE	100
+
+extern Size CachedBufShmemSize(void);
+extern void InitCachedBufTable(int size);
+extern void CachedBufTableInsert(RelFileNode *rnode, BlockNumber *blockNum,
+								 int buf_id);
+extern int CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				  int *forknum_indexes, BlockNumber *firstDelBlock,
+				  int *buf_id_array, int size);
+extern void CachedBufTableDelete(RelFileNode *rnode,  BlockNumber *blockNum,
+								 int buf_id);
+
 #endif							/* BUFMGR_INTERNALS_H */
-- 
1.8.3.1

#9k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#8)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi,

I have rebased the patch to keep the CFbot happy.
Apparently, in the previous patch there was a possibility of infinite loop
when allocating buffers, so I fixed that part and also removed some whitespaces.

Kindly check the attached V6 patch.
Any thoughts on this?

Regards,
Kirk Jamison

Attachments:

v6-Optimize-dropping-of-relation-buffers-using-dlist.patchapplication/octet-stream; name=v6-Optimize-dropping-of-relation-buffers-using-dlist.patchDownload
From 3ae203671208a60b04122ac22f3ad54c797dfb86 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Tue, 12 Nov 2019 08:55:00 +0000
Subject: [PATCH] Optimize dropping of relation buffers using dlist

Whenever buffer manager is called by INSERT, SELECT,
UPDATE, DELETE, TRUNCATE, VACUUM commands, the list
of buffers is also updated in an auxiliary hash table.
Cached buffer hash table is a new auxiliary hash table
structure added to the shared memory. Each hash table
entry contains key (relation file node and modulo of
block number) and buffer ID which points to the head
entry of a doubly-linked list.  The cached buffer list
links the cached pages of a target relation to a
doubly-linked list and provides the ability to improve
performance of invalidationg relation buffers by
simply scanning the list, rather than scanning the
entire shared_buffers.
---
 src/backend/storage/buffer/Makefile     |   1 +
 src/backend/storage/buffer/buf_init.c   |   6 +
 src/backend/storage/buffer/bufmgr.c     |  87 ++---
 src/backend/storage/buffer/cached_buf.c | 571 ++++++++++++++++++++++++++++++++
 src/include/storage/buf_internals.h     |  14 +
 5 files changed, 638 insertions(+), 41 deletions(-)
 create mode 100644 src/backend/storage/buffer/cached_buf.c

diff --git a/src/backend/storage/buffer/Makefile b/src/backend/storage/buffer/Makefile
index fd7c40d..3662349 100644
--- a/src/backend/storage/buffer/Makefile
+++ b/src/backend/storage/buffer/Makefile
@@ -16,6 +16,7 @@ OBJS = \
 	buf_init.o \
 	buf_table.o \
 	bufmgr.o \
+	cached_buf.o \
 	freelist.o \
 	localbuf.o
 
diff --git a/src/backend/storage/buffer/buf_init.c b/src/backend/storage/buffer/buf_init.c
index af62d48..0d17cfd 100644
--- a/src/backend/storage/buffer/buf_init.c
+++ b/src/backend/storage/buffer/buf_init.c
@@ -145,6 +145,9 @@ InitBufferPool(void)
 	/* Init other shared buffer-management stuff */
 	StrategyInitialize(!foundDescs);
 
+	/* Init cached buffer hash table and related shmem data structures */
+	InitCachedBufTable(NBuffers);
+
 	/* Initialize per-backend file flush context */
 	WritebackContextInit(&BackendWritebackContext,
 						 &backend_flush_after);
@@ -188,5 +191,8 @@ BufferShmemSize(void)
 	/* size of checkpoint sort array in bufmgr.c */
 	size = add_size(size, mul_size(NBuffers, sizeof(CkptSortItem)));
 
+	/* size of cached buffer shmem data structures */
+	size = add_size(size, CachedBufShmemSize());
+
 	return size;
 }
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index aba3960..c8d1a00 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -1319,10 +1319,12 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	if (oldPartitionLock != NULL)
 	{
+		CachedBufTableDelete(&oldTag.rnode, &oldTag.blockNum, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
 		if (oldPartitionLock != newPartitionLock)
 			LWLockRelease(oldPartitionLock);
 	}
+	CachedBufTableInsert(&newTag.rnode, &newTag.blockNum, buf->buf_id);
 
 	LWLockRelease(newPartitionLock);
 
@@ -1432,7 +1434,10 @@ retry:
 	 * Remove the buffer from the lookup hashtable, if it was in there.
 	 */
 	if (oldFlags & BM_TAG_VALID)
+	{
+		CachedBufTableDelete(&oldTag.rnode, &oldTag.blockNum, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
+	}
 
 	/*
 	 * Done with mapping lock.
@@ -2926,11 +2931,6 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
@@ -2938,59 +2938,64 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
-	int			j;
+	int			nbufs;
+	int			buf_id_array[BUF_ID_ARRAY_SIZE];
+	int			forknum_indexes[BUF_ID_ARRAY_SIZE];
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
 	{
 		if (rnode.backend == MyBackendId)
 		{
-			for (j = 0; j < nforks; j++)
-				DropRelFileNodeLocalBuffers(rnode.node, forkNum[j],
-											firstDelBlock[j]);
+			for (i = 0; i < nforks; i++)
+				DropRelFileNodeLocalBuffers(rnode.node, forkNum[i],
+											firstDelBlock[i]);
 		}
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	do
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		nbufs = CachedBufLookup(rnode.node, forkNum, nforks,
+								forknum_indexes, firstDelBlock,
+								buf_id_array, lengthof(buf_id_array));
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+		for (i = 0; i < nbufs; i++)
+		{
+			BufferDesc	*bufHdr = GetBufferDescriptor(buf_id_array[i]);
+			uint32		buf_state;
+			int		index = forknum_indexes[i];
 
-		buf_state = LockBufHdr(bufHdr);
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag
+			 * before we attempt to lock the buffer; this saves a lot of
+			 * lock acquisitions in typical cases.  It should be safe
+			 * because the caller must have AccessExclusiveLock on the
+			 * relation, or some other reason to be certain that no one is
+			 * loading new pages of the rel into the buffer pool.
+			 * (Otherwise we might well miss such pages entirely.)
+			 * Therefore, while the tag might be changing while we look at
+			 * it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible,
+			 * and false positives are safe because we'll recheck after
+			 * getting the buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but
+			 * the incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
+
+			buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
-		{
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
-			{
+				bufHdr->tag.forkNum == forkNum[index] &&
+				bufHdr->tag.blockNum >= firstDelBlock[index])
 				InvalidateBuffer(bufHdr); /* releases spinlock */
-				break;
-			}
+			else
+				UnlockBufHdr(bufHdr, buf_state);
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
-	}
+	} while (nbufs == lengthof(buf_id_array));
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/buffer/cached_buf.c b/src/backend/storage/buffer/cached_buf.c
new file mode 100644
index 0000000..8af05f2
--- /dev/null
+++ b/src/backend/storage/buffer/cached_buf.c
@@ -0,0 +1,571 @@
+/*-------------------------------------------------------------------------
+ *
+ * cached_buf.c
+ *	  routines for mapping relations to the indexes of auxillary cached
+ *	  buffers.
+ *
+ * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ *	  src/backend/storage/buffer/cached_buf.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "storage/bufmgr.h"
+#include "storage/buf_internals.h"
+
+
+/*
+ * Each relation and its buffer information are cached in a hash table
+ * located in shared memory. These cached buffers are chained together
+ * in a doubly-linked list.
+ */
+#define NUM_MAP_PARTITIONS_FOR_REL	128	/* relation-level */
+#define NUM_MAP_PARTITIONS_IN_REL	4	/* block-level */
+#define NUM_MAP_PARTITIONS \
+	(NUM_MAP_PARTITIONS_FOR_REL * NUM_MAP_PARTITIONS_IN_REL)
+#define CACHED_BUF_END_OF_LIST	(-1) /* end of doubly linked list */
+
+/* hash table key */
+typedef struct CachedBufTag
+{
+	RelFileNode		rnode;		/* relation */
+	int			modOfBlockNum;	/* modulo of BlockNumber*/
+} CachedBufTag;
+
+/* entry to the cached buffer hash table */
+typedef struct CachedBufEnt
+{
+	CachedBufTag	key;	/* hash table key */
+	int		head;	/* index of dlist */
+} CachedBufEnt;
+
+/* entry to the doubly-linked list */
+typedef struct BufDlistEnt
+{
+	int		prev;
+	int		next;
+} BufDlistEnt;
+
+/* Locks to be used for the hash table operations */
+typedef struct CombinedLock
+{
+	volatile bool	flag;	/* flag for lookup operation */
+	slock_t		spinLock;	/* spinlock to protect the flag */
+	LWLock		lwLock;	/* LWLock used in lookup */
+} CombinedLock;
+
+/* lock for cached buffer hash table */
+typedef struct CachedBufTableLock
+{
+	CombinedLock	cacheLock[NUM_MAP_PARTITIONS];
+	int		cacheTrancheId;
+} CachedBufTableLock;
+
+static HTAB *CachedBufHash = NULL;
+static BufDlistEnt *BufDlistEntArray = NULL;
+static CachedBufTableLock *CachedBufLockArray = NULL;
+static void CombinedLockInitialize(CombinedLock *lock, int trancheId);
+static uint32 CachedBufTableHashCode(CachedBufTag *tag);
+static CombinedLock *GetCachedBufPartitionLock(CachedBufTag *tag,
+											   uint32 *hashcode);
+static void CombinedLockAcquireSpinLock(CombinedLock *lock);
+static void CombinedLockAcquireLWLock(CombinedLock *lock);
+static void CombinedLockReleaseSpinLock(CombinedLock *lock);
+static void CombinedLockReleaseLWLock(CombinedLock *lock);
+static inline BufDlistEnt *cb_dlist_entry(int buf_id);
+static inline BufDlistEnt *cb_dlist_next(BufDlistEnt *entry);
+static inline BufDlistEnt *cb_dlist_prev(BufDlistEnt *entry);
+static inline bool cb_dlist_is_empty(CachedBufEnt *hash_entry);
+static inline void cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp);
+
+/*
+ * CachedBufShmemSize
+ * 		Estimate space needed for mapping cached buffer hash table
+ *
+ * 		size of lookup table is the desired hash table size
+ *		(possibly more than NBuffers)
+ */
+Size
+CachedBufShmemSize(void)
+{
+	Size		size = 0;
+
+	/* size of cached buffer lookup table */
+	size = add_size(size, hash_estimate_size(NBuffers +
+											 NUM_MAP_PARTITIONS,
+											 sizeof(CachedBufEnt)));
+
+	/* size of cached buffer dlist entry array */
+	size = add_size(size, mul_size(NBuffers, sizeof(BufDlistEnt)));
+
+	/* size of locks */
+	size = add_size(size, mul_size(NBuffers,
+								   sizeof(CachedBufTableLock)));
+
+	return size;
+}
+
+/* Initialize spinlock and LWLock for cached buffer hash table */
+void
+CombinedLockInitialize(CombinedLock *lock, int trancheId)
+{
+	lock->flag = false;
+	SpinLockInit(&lock->spinLock);
+	LWLockInitialize(&lock->lwLock, trancheId);
+}
+
+/*
+ * InitCachedBufTable
+ *      Initialize the cached buffer hash table and related data
+ *      structures at shared memory initialization.
+ */
+void
+InitCachedBufTable(int size)
+{
+	HASHCTL		info;
+	bool		foundList;
+	bool		foundLock;
+	int		i;
+
+	info.keysize = sizeof(CachedBufTag);
+	info.entrysize = sizeof(CachedBufEnt);
+	info.num_partitions = NUM_MAP_PARTITIONS;
+
+	CachedBufHash = ShmemInitHash("Cached Buffer Lookup Table",
+								  size + NUM_MAP_PARTITIONS,
+								  size + NUM_MAP_PARTITIONS,
+								  &info,
+								  HASH_ELEM | HASH_BLOBS |
+								  HASH_PARTITION);
+
+	BufDlistEntArray = ShmemInitStruct("buffer dlist entry array",
+									   size * sizeof(BufDlistEnt),
+									   &foundList);
+
+	CachedBufLockArray = (CachedBufTableLock *)
+			ShmemInitStruct("cached buffer hash partition lock",
+							size * sizeof(CachedBufTableLock),
+							&foundLock);
+
+	if (!foundList && !foundLock)
+	{
+		CachedBufLockArray->cacheTrancheId = LWLockNewTrancheId();
+
+		for (i = 0; i < NUM_MAP_PARTITIONS; i++)
+			CombinedLockInitialize(&CachedBufLockArray->cacheLock[i],
+								   CachedBufLockArray->cacheTrancheId);
+	}
+	LWLockRegisterTranche(CachedBufLockArray->cacheTrancheId,
+						  "cached_buf_tranche_id");
+
+}
+
+ /*
+  * CachedBufTableHashCode
+  *      Compute the hash code associated with tag
+  *
+  * This must be passed to the insert/lookup/delete routines along with the
+  * tag. We do it like this because the callers need to know the hash code
+  * to determine which partition to lock, and we don't want to do the  hash
+  * computation twice (hash_any is a bit slow).
+  */
+static uint32
+CachedBufTableHashCode(CachedBufTag *tag)
+{
+	return get_hash_value(CachedBufHash, (void *) tag);
+}
+
+/*
+ * GetCachedBufPartitionLock
+ * 		Get lock instance for partition of cached buffer lookup table
+ */
+static CombinedLock *
+GetCachedBufPartitionLock(CachedBufTag *tag, uint32 *hashcode)
+{
+	*hashcode = CachedBufTableHashCode(tag);
+
+	return &CachedBufLockArray->cacheLock[*hashcode % NUM_MAP_PARTITIONS];
+}
+
+/*
+ * CombinedLockAcquireSpinLock
+ *		Acquire spinlock to proceed to inserting/deleting hash
+ *		table entries
+ *
+ * This function waits for a lookup process (if flag is true) to finish,
+ * before reacquring a spinlock.
+ */
+void
+CombinedLockAcquireSpinLock(CombinedLock *lock)
+{
+	SpinLockAcquire(&lock->spinLock);
+
+	while (lock->flag)
+	{
+		SpinLockRelease(&lock->spinLock);
+		LWLockAcquire(&lock->lwLock, LW_EXCLUSIVE);
+		LWLockRelease(&lock->lwLock);
+		SpinLockAcquire(&lock->spinLock);
+	}
+}
+
+/*
+ * CombinedLockAcquireLWLock
+ *		Get LWLock instance for partition of cached buffer lookup table
+ *
+ * To prevent other backend processes from modifying the lookup table,
+ * this function sets the CombinedLock's flag to true.
+ */
+void
+CombinedLockAcquireLWLock(CombinedLock *lock)
+{
+	LWLockAcquire(&lock->lwLock, LW_EXCLUSIVE);
+	SpinLockAcquire(&lock->spinLock);
+	lock->flag = true;
+	SpinLockRelease(&lock->spinLock);
+}
+
+/*
+ * CombinedLockReleaseSpinLock
+ *		Release the spinlock used in insert/delete functions
+ */
+void
+CombinedLockReleaseSpinLock(CombinedLock *lock)
+{
+	SpinLockRelease(&lock->spinLock);
+}
+
+/*
+ * CombinedLockReleaseSpinLock
+ *		Release the LWLock used in lookup function after changing the
+ *		CombinedLock's flag to false.
+ */
+void
+CombinedLockReleaseLWLock(CombinedLock *lock)
+{
+	lock->flag = false;
+	LWLockRelease(&lock->lwLock);
+}
+
+/*
+ * CachedBufTableInsert
+ *		Insert a hash table entry for given tag and buffer ID
+ *
+ * If a hash entry for the tag already exists, only buffer ID is inserted.
+ * Chain the buffer ID to the doubly-linked list.
+ *
+ * This function must hold spin lock.
+ */
+void
+CachedBufTableInsert(RelFileNode *rnode, BlockNumber *blockNum, int buf_id)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	cb_tag.rnode = *rnode;
+	cb_tag.modOfBlockNum = *blockNum % NUM_MAP_PARTITIONS_IN_REL;
+
+	combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+	CombinedLockAcquireSpinLock(combined_lock);
+
+	/* look up or create a hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &cb_tag,
+									hashcode,
+									HASH_ENTER,
+									&found);
+
+	/* If not found, initialize linked list */
+	if (!found)
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+
+	cb_dlist_push_head(hash_entry, buf_id);
+
+	CombinedLockReleaseSpinLock(combined_lock);
+}
+
+/*
+ * CachedBufLookup
+ *		Lookup the buffers for the given tag in the cached buffer
+ *		hash table. Insert the buffer ID to the array of buffer
+ *		IDs, and return the total number of buffers in the array.
+ *
+ * This function must hold exclusive LWLock for tag's partition.
+ */
+int
+CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				int *forknum_indexes, BlockNumber *firstDelBlock,
+				int *buf_id_array, int size)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	CachedBufEnt	temp_hash_entry;
+	BufDlistEnt	*curr_entry = NULL;
+	int		mod;
+	int		i;
+	int		curr_buf_id;
+	int		new_curr_buf_id;
+	int		count = 0;
+	bool		target_buf = false;
+
+	for (mod = 0; mod < NUM_MAP_PARTITIONS_IN_REL; mod++)
+	{
+		cb_tag.rnode = rnode;
+		cb_tag.modOfBlockNum = mod;
+
+		combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+		CombinedLockAcquireLWLock(combined_lock);
+
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) &cb_tag,
+										hashcode,
+										HASH_FIND,
+										NULL);
+
+		if (!hash_entry)
+		{
+			CombinedLockReleaseLWLock(combined_lock);
+			continue;
+		}
+
+		/* Initial temporary dlist */
+		temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+
+		/* When traversing the main dlist, start from head */
+		curr_buf_id = hash_entry->head;
+
+		while(curr_buf_id != CACHED_BUF_END_OF_LIST && count < size)
+		{
+			BufferDesc *bufHdr = GetBufferDescriptor(curr_buf_id);
+			curr_entry = BufDlistEntArray + curr_buf_id;
+			new_curr_buf_id = curr_entry->next;
+
+			/* Check if it's our target buffer */
+			for (i = 0; i < nforks; i++)
+			{
+				if (bufHdr->tag.forkNum != forkNum[i])
+					continue;
+				else
+				{
+					if (bufHdr->tag.blockNum >= firstDelBlock[i])
+						target_buf = true;
+					break;
+				}
+			}
+
+			if (target_buf)
+			{
+				forknum_indexes[count] = i;
+				buf_id_array[count] = curr_buf_id;
+				++count;
+			}
+			else
+			{
+				/*
+				* It's not the target buffer. Remove the current buffer ID
+				* from the current list of target buffer IDs and store it
+				* to a temporary list.
+				*/
+				cb_dlist_delete(hash_entry, curr_buf_id);
+				cb_dlist_push_head(&temp_hash_entry, curr_buf_id);
+			}
+			/* Move current pointer to next */
+			curr_buf_id = new_curr_buf_id;
+		}
+
+		/* Check if main dlist is now empty */
+		if (cb_dlist_is_empty(hash_entry))
+		{
+			hash_entry->head = temp_hash_entry.head;
+			temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+		}
+
+		/* If we have a temporary dlist, append it to the main dlist */
+		if (!cb_dlist_is_empty(hash_entry) &&
+			!cb_dlist_is_empty(&temp_hash_entry))
+			cb_dlist_combine(hash_entry, &temp_hash_entry);
+
+		CombinedLockReleaseLWLock(combined_lock);
+	}
+
+	return count;
+}
+
+/*
+ * CachedBufTableDelete
+ *		Unlink the buffer ID from the doubly-linked list, then remove
+ *		the hash entry for the given tag if its list is empty.
+ *
+ * This function must hold spin lock.
+ */
+void
+CachedBufTableDelete(RelFileNode *rnode, BlockNumber *blockNum, int buf_id)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	cb_tag.rnode = *rnode;
+	cb_tag.modOfBlockNum = *blockNum % NUM_MAP_PARTITIONS_IN_REL;
+
+	combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+	CombinedLockAcquireSpinLock(combined_lock);
+
+	/* look up hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &cb_tag,
+									hashcode,
+									HASH_FIND,
+									&found);
+
+	if (!found)		/* tag not found, nothing to do */
+	{
+		CombinedLockReleaseSpinLock(combined_lock);
+		return;
+	}
+
+	cb_dlist_delete(hash_entry, buf_id);
+
+	/*
+	 * If there's no more cached elements for the given tag,
+	 * remove the hash entry.
+	 */
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) &cb_tag,
+										hashcode,
+										HASH_REMOVE,
+										NULL);
+
+		if (!hash_entry)
+			elog(ERROR, "cached buffer hash table corrupted");
+	}
+
+	CombinedLockReleaseSpinLock(combined_lock);
+}
+
+/*
+ * inline functions for the doubly-linked list of cached buffers
+ */
+/* Return the dlist entry */
+static inline BufDlistEnt *
+cb_dlist_entry(int buf_id)
+{
+	return BufDlistEntArray + buf_id;
+}
+
+/* Return the next entry */
+static inline BufDlistEnt *
+cb_dlist_next(BufDlistEnt *entry)
+{
+	return entry->next == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->next);
+}
+
+/* Return the prev entry */
+static inline BufDlistEnt *
+cb_dlist_prev(BufDlistEnt *entry)
+{
+	return entry->prev == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->prev);
+}
+
+/* Return if dlist is empty */
+static inline bool
+cb_dlist_is_empty(CachedBufEnt *hash_entry)
+{
+	return hash_entry->head == CACHED_BUF_END_OF_LIST;
+}
+
+/* Push to head of dlist */
+static inline void
+cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*new_entry = cb_dlist_entry(buf_id);
+	new_entry->next = hash_entry->head;
+
+	if (cb_dlist_is_empty(hash_entry))
+		new_entry->prev = buf_id;
+	else
+	{
+		BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+		new_entry->prev = head_entry->prev;
+		cb_dlist_next(new_entry)->prev = buf_id;
+	}
+	hash_entry->head = buf_id;
+}
+
+/* Remove the buffer ID from dlist */
+static inline void
+cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*curr_entry = cb_dlist_entry(buf_id);
+	BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+	BufDlistEnt	*tail_entry = cb_dlist_entry(head_entry->prev);
+	BufDlistEnt	*next_entry = cb_dlist_next(curr_entry);
+	BufDlistEnt	*prev_entry = cb_dlist_prev(curr_entry);
+
+	/* If only one entry is in the list */
+	if (head_entry == tail_entry)
+	{
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+		return;
+	}
+
+	/*
+	 * If there is a next entry, update its prev field.
+	 * Otherwise, current entry is at tail (but not at head),
+	 * so update the new tail.
+	 */
+	if (next_entry != NULL)
+		next_entry->prev = curr_entry->prev;
+	else
+		head_entry->prev = curr_entry->prev;
+
+	/*
+	 * If the previous entry is not the tail entry, update its
+	 * next field. Otherwise, current entry is at head (but not
+	 * at tail). Update the new head entry and its tail pointer.
+	 */
+	if (prev_entry != tail_entry)
+		prev_entry->next = curr_entry->next;
+	else
+		hash_entry->head = curr_entry->next;
+}
+
+/* Append head of temporary dlist to main dlist */
+static inline void
+cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp)
+{
+	BufDlistEnt	*main_head_entry = cb_dlist_entry(main->head);
+	BufDlistEnt	*temp_head_entry = cb_dlist_entry(temp->head);
+	int	main_tail = main_head_entry->prev;
+	int	temp_tail = temp_head_entry->prev;
+	BufDlistEnt	*main_tail_entry = cb_dlist_entry(main_tail);
+
+	/* Append the temporary dlist to main dlist */
+	main_tail_entry->next = temp->head;
+	temp_head_entry->prev = main_tail;
+	main_head_entry->prev = temp_tail;
+
+	/* Clear the head of temporary dlist */
+	temp->head = CACHED_BUF_END_OF_LIST;
+}
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index bf3b8ad..a7c0b18 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -337,4 +337,18 @@ extern void DropRelFileNodeLocalBuffers(RelFileNode rnode, ForkNumber forkNum,
 extern void DropRelFileNodeAllLocalBuffers(RelFileNode rnode);
 extern void AtEOXact_LocalBuffers(bool isCommit);
 
+/* in cached_buf.c */
+/* size of array of linked buffer ID's of target relation */
+#define BUF_ID_ARRAY_SIZE	100
+
+extern Size CachedBufShmemSize(void);
+extern void InitCachedBufTable(int size);
+extern void CachedBufTableInsert(RelFileNode *rnode, BlockNumber *blockNum,
+								 int buf_id);
+extern int CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				  int *forknum_indexes, BlockNumber *firstDelBlock,
+				  int *buf_id_array, int size);
+extern void CachedBufTableDelete(RelFileNode *rnode,  BlockNumber *blockNum,
+								 int buf_id);
+
 #endif							/* BUFMGR_INTERNALS_H */
-- 
1.8.3.1

#10Robert Haas
robertmhaas@gmail.com
In reply to: k.jamison@fujitsu.com (#9)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Feb 4, 2020 at 4:57 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Kindly check the attached V6 patch.
Any thoughts on this?

Unfortunately, I don't have time for detailed review of this. I am
suspicious that there are substantial performance regressions that you
just haven't found yet. I would not take the position that this is a
completely hopeless approach, or anything like that, but neither would
I conclude that the tests shown so far are anywhere near enough to be
confident that there are no problems.

Also, systems with very large shared_buffers settings are becoming
more common, and probably will continue to become more common, so I
don't think we can dismiss that as an edge case any more. People don't
want to run with an 8GB cache on a 1TB server.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#11k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Robert Haas (#10)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi,

I know this might already be late at end of CommitFest, but attached
is the latest version of the patch. The previous version only includes buffer
invalidation improvement for VACUUM. The new patch adds the same
routine for TRUNCATE WAL replay.

In summary, this patch aims to improve the buffer invalidation process
of VACUUM and TRUNCATE. Although it may not be a common use
case, our customer uses these commands a lot. Recovery and WAL
replay of these commands can take time depending on the size of
database buffers. So this patch optimizes that using the newly-added
auxiliary cache and doubly-linked list on the shared memory, so that
we don't need to scan the shared buffers anymore.

As for the performance and how it affects the read-only workloads.
Using pgbench, I've kept the overload to a minimum, less than 1%.
I'll post follow-up results.

Although the additional hash table utilizes shared memory, there's
a significant performance gain for both TRUNCATE and VACUUM
from execution to recovery.

Regards,
Kirk Jamison

Attachments:

v7-Optimize-dropping-of-relation-buffers-using-dlist.patchapplication/octet-stream; name=v7-Optimize-dropping-of-relation-buffers-using-dlist.patchDownload
From 8698283b06f488bdcc655374b172f01dece6cdc2 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 13 Mar 2020 00:47:24 +0000
Subject: [PATCH] Optimize dropping of relation buffers using dlist

The invalidation of relation buffers does not depend
on the size of the shared_buffers anymore. This
improves the buffer invalidation process of VACUUM and
TRUNCATE.

Cached buffer hash table is a new auxiliary hash table
structure added to the shared memory. Each hash table
entry contains key (relation file node and modulo of
block number) and buffer ID which points to the head
entry of a doubly-linked list. The cached buffer list
links the to-be-invalidated buffers of a relation to a
doubly-linked list. This significantly improves the
performance of invalidating relation buffers in
DropRelFileNodeBuffers and DropRelFileNodeBuffersAll,
by simply scanning the list rather than scanning the
entire shared_buffers.
---
 src/backend/storage/buffer/Makefile     |   1 +
 src/backend/storage/buffer/buf_init.c   |   6 +
 src/backend/storage/buffer/bufmgr.c     | 161 ++++----
 src/backend/storage/buffer/cached_buf.c | 638 ++++++++++++++++++++++++++++++++
 src/include/storage/buf_internals.h     |  16 +
 5 files changed, 748 insertions(+), 74 deletions(-)
 create mode 100644 src/backend/storage/buffer/cached_buf.c

diff --git a/src/backend/storage/buffer/Makefile b/src/backend/storage/buffer/Makefile
index fd7c40d..3662349 100644
--- a/src/backend/storage/buffer/Makefile
+++ b/src/backend/storage/buffer/Makefile
@@ -16,6 +16,7 @@ OBJS = \
 	buf_init.o \
 	buf_table.o \
 	bufmgr.o \
+	cached_buf.o \
 	freelist.o \
 	localbuf.o
 
diff --git a/src/backend/storage/buffer/buf_init.c b/src/backend/storage/buffer/buf_init.c
index af62d48..0d17cfd 100644
--- a/src/backend/storage/buffer/buf_init.c
+++ b/src/backend/storage/buffer/buf_init.c
@@ -145,6 +145,9 @@ InitBufferPool(void)
 	/* Init other shared buffer-management stuff */
 	StrategyInitialize(!foundDescs);
 
+	/* Init cached buffer hash table and related shmem data structures */
+	InitCachedBufTable(NBuffers);
+
 	/* Initialize per-backend file flush context */
 	WritebackContextInit(&BackendWritebackContext,
 						 &backend_flush_after);
@@ -188,5 +191,8 @@ BufferShmemSize(void)
 	/* size of checkpoint sort array in bufmgr.c */
 	size = add_size(size, mul_size(NBuffers, sizeof(CkptSortItem)));
 
+	/* size of cached buffer shmem data structures */
+	size = add_size(size, CachedBufShmemSize());
+
 	return size;
 }
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e05e2b3..0b46384 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -1266,10 +1266,12 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	if (oldPartitionLock != NULL)
 	{
+		CachedBufTableDelete(&oldTag.rnode, &oldTag.blockNum, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
 		if (oldPartitionLock != newPartitionLock)
 			LWLockRelease(oldPartitionLock);
 	}
+	CachedBufTableInsert(&newTag.rnode, &newTag.blockNum, buf->buf_id);
 
 	LWLockRelease(newPartitionLock);
 
@@ -1379,7 +1381,10 @@ retry:
 	 * Remove the buffer from the lookup hashtable, if it was in there.
 	 */
 	if (oldFlags & BM_TAG_VALID)
+	{
+		CachedBufTableDelete(&oldTag.rnode, &oldTag.blockNum, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
+	}
 
 	/*
 	 * Done with mapping lock.
@@ -2873,71 +2878,71 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
 DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
-	int			i;
-	int			j;
+	int			i,
+				nbufs,
+				buf_id_array[BUF_ID_ARRAY_SIZE],
+				forknum_indexes[BUF_ID_ARRAY_SIZE];
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
 	{
 		if (rnode.backend == MyBackendId)
 		{
-			for (j = 0; j < nforks; j++)
-				DropRelFileNodeLocalBuffers(rnode.node, forkNum[j],
-											firstDelBlock[j]);
+			for (i = 0; i < nforks; i++)
+				DropRelFileNodeLocalBuffers(rnode.node, forkNum[i],
+											firstDelBlock[i]);
 		}
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	do
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		nbufs = CachedBufLookup(rnode.node, forkNum, nforks,
+								forknum_indexes, firstDelBlock,
+								buf_id_array, lengthof(buf_id_array));
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+		for (i = 0; i < nbufs; i++)
+		{
+			BufferDesc	*bufHdr = GetBufferDescriptor(buf_id_array[i]);
+			uint32		buf_state;
+			int		index = forknum_indexes[i];
 
-		buf_state = LockBufHdr(bufHdr);
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag
+			 * before we attempt to lock the buffer; this saves a lot of
+			 * lock acquisitions in typical cases.  It should be safe
+			 * because the caller must have AccessExclusiveLock on the
+			 * relation, or some other reason to be certain that no one is
+			 * loading new pages of the rel into the buffer pool.
+			 * (Otherwise we might well miss such pages entirely.)
+			 * Therefore, while the tag might be changing while we look at
+			 * it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible,
+			 * and false positives are safe because we'll recheck after
+			 * getting the buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but
+			 * the incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
+
+			buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
-		{
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
-			{
+				bufHdr->tag.forkNum == forkNum[index] &&
+				bufHdr->tag.blockNum >= firstDelBlock[index])
 				InvalidateBuffer(bufHdr); /* releases spinlock */
-				break;
-			}
+			else
+				UnlockBufHdr(bufHdr, buf_state);
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
-	}
+	} while (nbufs == lengthof(buf_id_array));
 }
 
 /* ---------------------------------------------------------------------
@@ -2953,7 +2958,9 @@ void
 DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 {
 	int			i,
-				n = 0;
+				n = 0,
+				nbufs,
+				buf_id_array[BUF_ID_ARRAY_SIZE];
 	RelFileNode *nodes;
 	bool		use_bsearch;
 
@@ -2996,47 +3003,53 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	if (use_bsearch)
 		pg_qsort(nodes, n, sizeof(RelFileNode), rnode_comparator);
 
-	for (i = 0; i < NBuffers; i++)
+	do
 	{
-		RelFileNode *rnode = NULL;
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
-
-		/*
-		 * As in DropRelFileNodeBuffers, an unlocked precheck should be safe
-		 * and saves some cycles.
-		 */
+		nbufs = CachedBufLookupAll(nodes, nnodes, buf_id_array,
+								   lengthof(buf_id_array));
 
-		if (!use_bsearch)
+		for (i = 0; i < nbufs; i++)
 		{
-			int			j;
+			RelFileNode *rnode = NULL;
+			BufferDesc	*bufHdr = GetBufferDescriptor(buf_id_array[i]);
+			uint32		buf_state;
 
-			for (j = 0; j < n; j++)
+			/*
+			* As in DropRelFileNodeBuffers, an unlocked precheck should be safe
+			* and saves some cycles.
+			*/
+
+			if (!use_bsearch)
 			{
-				if (RelFileNodeEquals(bufHdr->tag.rnode, nodes[j]))
+				int			j;
+
+				for (j = 0; j < n; j++)
 				{
-					rnode = &nodes[j];
-					break;
+					if (RelFileNodeEquals(bufHdr->tag.rnode, nodes[j]))
+					{
+						rnode = &nodes[j];
+						break;
+					}
 				}
 			}
-		}
-		else
-		{
-			rnode = bsearch((const void *) &(bufHdr->tag.rnode),
-							nodes, n, sizeof(RelFileNode),
-							rnode_comparator);
-		}
+			else
+			{
+				rnode = bsearch((const void *) &(bufHdr->tag.rnode),
+								nodes, n, sizeof(RelFileNode),
+								rnode_comparator);
+			}
 
-		/* buffer doesn't belong to any of the given relfilenodes; skip it */
-		if (rnode == NULL)
-			continue;
+			/* buffer doesn't belong to any of the given relfilenodes; skip it */
+			if (rnode == NULL)
+				continue;
 
-		buf_state = LockBufHdr(bufHdr);
-		if (RelFileNodeEquals(bufHdr->tag.rnode, (*rnode)))
-			InvalidateBuffer(bufHdr);	/* releases spinlock */
-		else
-			UnlockBufHdr(bufHdr, buf_state);
-	}
+			buf_state = LockBufHdr(bufHdr);
+			if (RelFileNodeEquals(bufHdr->tag.rnode, (*rnode)))
+				InvalidateBuffer(bufHdr);	/* releases spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);
+		}
+	} while (nbufs == lengthof(buf_id_array));
 
 	pfree(nodes);
 }
diff --git a/src/backend/storage/buffer/cached_buf.c b/src/backend/storage/buffer/cached_buf.c
new file mode 100644
index 0000000..1e38087
--- /dev/null
+++ b/src/backend/storage/buffer/cached_buf.c
@@ -0,0 +1,638 @@
+/*-------------------------------------------------------------------------
+ *
+ * cached_buf.c
+ *	  routines for mapping relations to the indexes of auxillary cached
+ *	  buffers.
+ *
+ * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ *	  src/backend/storage/buffer/cached_buf.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "storage/bufmgr.h"
+#include "storage/buf_internals.h"
+
+
+/*
+ * Each relation and its buffer information are cached in a hash table
+ * located in shared memory. These cached buffers are chained together
+ * in a doubly-linked list.
+ */
+#define NUM_MAP_PARTITIONS_FOR_REL	128	/* relation-level */
+#define NUM_MAP_PARTITIONS_IN_REL	4	/* block-level */
+#define NUM_MAP_PARTITIONS \
+	(NUM_MAP_PARTITIONS_FOR_REL * NUM_MAP_PARTITIONS_IN_REL)
+#define CACHED_BUF_END_OF_LIST	(-1) /* end of doubly linked list */
+
+/* hash table key */
+typedef struct CachedBufTag
+{
+	RelFileNode		rnode;		/* relation */
+	int			modOfBlockNum;	/* modulo of BlockNumber*/
+} CachedBufTag;
+
+/* entry to the cached buffer hash table */
+typedef struct CachedBufEnt
+{
+	CachedBufTag	key;	/* hash table key */
+	int		head;	/* index of dlist */
+} CachedBufEnt;
+
+/* entry to the doubly-linked list */
+typedef struct BufDlistEnt
+{
+	int		prev;
+	int		next;
+} BufDlistEnt;
+
+/* Locks to be used for the hash table operations */
+typedef struct CombinedLock
+{
+	volatile bool	flag;	/* flag for lookup operation */
+	slock_t		spinLock;	/* spinlock to protect the flag */
+	LWLock		lwLock;	/* LWLock used in lookup */
+} CombinedLock;
+
+/* lock for cached buffer hash table */
+typedef struct CachedBufTableLock
+{
+	CombinedLock	cacheLock[NUM_MAP_PARTITIONS];
+	int		cacheTrancheId;
+} CachedBufTableLock;
+
+static HTAB *CachedBufHash = NULL;
+static BufDlistEnt *BufDlistEntArray = NULL;
+static CachedBufTableLock *CachedBufLockArray = NULL;
+static void CombinedLockInitialize(CombinedLock *lock, int trancheId);
+static uint32 CachedBufTableHashCode(CachedBufTag *tag);
+static CombinedLock *GetCachedBufPartitionLock(CachedBufTag *tag,
+											   uint32 *hashcode);
+static void CombinedLockAcquireSpinLock(CombinedLock *lock);
+static void CombinedLockAcquireLWLock(CombinedLock *lock);
+static void CombinedLockReleaseSpinLock(CombinedLock *lock);
+static void CombinedLockReleaseLWLock(CombinedLock *lock);
+static inline BufDlistEnt *cb_dlist_entry(int buf_id);
+static inline BufDlistEnt *cb_dlist_next(BufDlistEnt *entry);
+static inline BufDlistEnt *cb_dlist_prev(BufDlistEnt *entry);
+static inline bool cb_dlist_is_empty(CachedBufEnt *hash_entry);
+static inline void cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp);
+
+/*
+ * CachedBufShmemSize
+ * 		Estimate space needed for mapping cached buffer hash table
+ *
+ * 		size of lookup table is the desired hash table size
+ *		(possibly more than NBuffers)
+ */
+Size
+CachedBufShmemSize(void)
+{
+	Size		size = 0;
+
+	/* size of cached buffer lookup table */
+	size = add_size(size, hash_estimate_size(NBuffers +
+											 NUM_MAP_PARTITIONS,
+											 sizeof(CachedBufEnt)));
+
+	/* size of cached buffer dlist entry array */
+	size = add_size(size, mul_size(NBuffers, sizeof(BufDlistEnt)));
+
+	/* size of locks */
+	size = add_size(size, mul_size(NBuffers,
+								   sizeof(CachedBufTableLock)));
+
+	return size;
+}
+
+/* Initialize spinlock and LWLock for cached buffer hash table */
+void
+CombinedLockInitialize(CombinedLock *lock, int trancheId)
+{
+	lock->flag = false;
+	SpinLockInit(&lock->spinLock);
+	LWLockInitialize(&lock->lwLock, trancheId);
+}
+
+/*
+ * InitCachedBufTable
+ *      Initialize the cached buffer hash table and related data
+ *      structures at shared memory initialization.
+ */
+void
+InitCachedBufTable(int size)
+{
+	HASHCTL		info;
+	bool		foundList,
+				foundLock;
+	int		i;
+
+	info.keysize = sizeof(CachedBufTag);
+	info.entrysize = sizeof(CachedBufEnt);
+	info.num_partitions = NUM_MAP_PARTITIONS;
+
+	CachedBufHash = ShmemInitHash("Cached Buffer Lookup Table",
+								  size + NUM_MAP_PARTITIONS,
+								  size + NUM_MAP_PARTITIONS,
+								  &info,
+								  HASH_ELEM | HASH_BLOBS |
+								  HASH_PARTITION);
+
+	BufDlistEntArray = ShmemInitStruct("buffer dlist entry array",
+									   size * sizeof(BufDlistEnt),
+									   &foundList);
+
+	CachedBufLockArray = (CachedBufTableLock *)
+			ShmemInitStruct("cached buffer hash partition lock",
+							size * sizeof(CachedBufTableLock),
+							&foundLock);
+
+	if (!foundList && !foundLock)
+	{
+		CachedBufLockArray->cacheTrancheId = LWLockNewTrancheId();
+
+		for (i = 0; i < NUM_MAP_PARTITIONS; i++)
+			CombinedLockInitialize(&CachedBufLockArray->cacheLock[i],
+								   CachedBufLockArray->cacheTrancheId);
+	}
+	LWLockRegisterTranche(CachedBufLockArray->cacheTrancheId,
+						  "cached_buf_tranche_id");
+
+}
+
+ /*
+  * CachedBufTableHashCode
+  *      Compute the hash code associated with tag
+  *
+  * This must be passed to the insert/lookup/delete routines along with the
+  * tag. We do it like this because the callers need to know the hash code
+  * to determine which partition to lock, and we don't want to do the  hash
+  * computation twice (hash_any is a bit slow).
+  */
+static uint32
+CachedBufTableHashCode(CachedBufTag *tag)
+{
+	return get_hash_value(CachedBufHash, (void *) tag);
+}
+
+/*
+ * GetCachedBufPartitionLock
+ * 		Get lock instance for partition of cached buffer lookup table
+ */
+static CombinedLock *
+GetCachedBufPartitionLock(CachedBufTag *tag, uint32 *hashcode)
+{
+	*hashcode = CachedBufTableHashCode(tag);
+
+	return &CachedBufLockArray->cacheLock[*hashcode % NUM_MAP_PARTITIONS];
+}
+
+/*
+ * CombinedLockAcquireSpinLock
+ *		Acquire spinlock to proceed to inserting/deleting hash
+ *		table entries
+ *
+ * This function waits for a lookup process (if flag is true) to finish,
+ * before reacquring a spinlock.
+ */
+void
+CombinedLockAcquireSpinLock(CombinedLock *lock)
+{
+	SpinLockAcquire(&lock->spinLock);
+
+	while (lock->flag)
+	{
+		SpinLockRelease(&lock->spinLock);
+		LWLockAcquire(&lock->lwLock, LW_EXCLUSIVE);
+		LWLockRelease(&lock->lwLock);
+		SpinLockAcquire(&lock->spinLock);
+	}
+}
+
+/*
+ * CombinedLockAcquireLWLock
+ *		Get LWLock instance for partition of cached buffer lookup table
+ *
+ * To prevent other backend processes from modifying the lookup table,
+ * this function sets the CombinedLock's flag to true.
+ */
+void
+CombinedLockAcquireLWLock(CombinedLock *lock)
+{
+	LWLockAcquire(&lock->lwLock, LW_EXCLUSIVE);
+	SpinLockAcquire(&lock->spinLock);
+	lock->flag = true;
+	SpinLockRelease(&lock->spinLock);
+}
+
+/*
+ * CombinedLockReleaseSpinLock
+ *		Release the spinlock used in insert/delete functions
+ */
+void
+CombinedLockReleaseSpinLock(CombinedLock *lock)
+{
+	SpinLockRelease(&lock->spinLock);
+}
+
+/*
+ * CombinedLockReleaseSpinLock
+ *		Release the LWLock used in lookup function after changing the
+ *		CombinedLock's flag to false.
+ */
+void
+CombinedLockReleaseLWLock(CombinedLock *lock)
+{
+	lock->flag = false;
+	LWLockRelease(&lock->lwLock);
+}
+
+/*
+ * CachedBufTableInsert
+ *		Insert a hash table entry for given tag and buffer ID
+ *
+ * If a hash entry for the tag already exists, only buffer ID is inserted.
+ * Chain the buffer ID to the doubly-linked list.
+ *
+ * This function must hold spin lock.
+ */
+void
+CachedBufTableInsert(RelFileNode *rnode, BlockNumber *blockNum, int buf_id)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	cb_tag.rnode = *rnode;
+	cb_tag.modOfBlockNum = *blockNum % NUM_MAP_PARTITIONS_IN_REL;
+
+	combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+	CombinedLockAcquireSpinLock(combined_lock);
+
+	/* look up or create a hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &cb_tag,
+									hashcode,
+									HASH_ENTER,
+									&found);
+
+	/* If not found, initialize linked list */
+	if (!found)
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+
+	cb_dlist_push_head(hash_entry, buf_id);
+
+	CombinedLockReleaseSpinLock(combined_lock);
+}
+
+/*
+ * CachedBufLookup
+ *		Lookup the buffers for the given tag in the cached buffer
+ *		hash table. Insert the buffer ID to the array of buffer
+ *		IDs, and return the total number of buffers to be invalidated.
+ *
+ * This function must hold exclusive LWLock for tag's partition.
+ */
+int
+CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				int *forknum_indexes, BlockNumber *firstDelBlock,
+				int *buf_id_array, int size)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry,
+					temp_hash_entry;
+	BufDlistEnt	*curr_entry = NULL;
+	int		mod,
+			i,
+			curr_buf_id,
+			new_curr_buf_id,
+			count = 0;
+	bool		target_buf = false;
+
+	for (mod = 0; mod < NUM_MAP_PARTITIONS_IN_REL; mod++)
+	{
+		cb_tag.rnode = rnode;
+		cb_tag.modOfBlockNum = mod;
+
+		combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+		CombinedLockAcquireLWLock(combined_lock);
+
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) &cb_tag,
+										hashcode,
+										HASH_FIND,
+										NULL);
+
+		if (!hash_entry)
+		{
+			CombinedLockReleaseLWLock(combined_lock);
+			continue;
+		}
+
+		/* Initial temporary dlist */
+		temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+
+		/* When traversing the main dlist, start from head */
+		curr_buf_id = hash_entry->head;
+
+		while(curr_buf_id != CACHED_BUF_END_OF_LIST && count < size)
+		{
+			BufferDesc *bufHdr = GetBufferDescriptor(curr_buf_id);
+			curr_entry = BufDlistEntArray + curr_buf_id;
+			new_curr_buf_id = curr_entry->next;
+
+			/* Check if it's our target buffer */
+			for (i = 0; i < nforks; i++)
+			{
+				if (bufHdr->tag.forkNum != forkNum[i])
+					continue;
+				else
+				{
+					if (bufHdr->tag.blockNum >= firstDelBlock[i])
+						target_buf = true;
+					break;
+				}
+			}
+
+			if (target_buf)
+			{
+				forknum_indexes[count] = i;
+				buf_id_array[count] = curr_buf_id;
+				++count;
+			}
+			else
+			{
+				/*
+				* It's not the target buffer. Remove the current buffer ID
+				* from the current list of target buffer IDs and store it
+				* to a temporary list.
+				*/
+				cb_dlist_delete(hash_entry, curr_buf_id);
+				cb_dlist_push_head(&temp_hash_entry, curr_buf_id);
+			}
+			/* Move current pointer to next */
+			curr_buf_id = new_curr_buf_id;
+		}
+
+		/* Check if main dlist is now empty */
+		if (cb_dlist_is_empty(hash_entry))
+		{
+			hash_entry->head = temp_hash_entry.head;
+			temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+		}
+
+		/* If we have a temporary dlist, append it to the main dlist */
+		if (!cb_dlist_is_empty(hash_entry) &&
+			!cb_dlist_is_empty(&temp_hash_entry))
+			cb_dlist_combine(hash_entry, &temp_hash_entry);
+
+		CombinedLockReleaseLWLock(combined_lock);
+	}
+
+	return count;
+}
+
+/*
+ * CachedBufLookupAll
+ *		Lookup all the buffers of specified relations in the cached buffer
+ *		hash table. Store all the buffer IDs to the array, and return
+ *		the total number of buffers to be invalidated.
+ *
+ * This function must hold exclusive LWLock for tag's partition.
+ */
+int
+CachedBufLookupAll(RelFileNode *rnode, int nnodes, int *buf_id_array,
+				   int size)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	BufDlistEnt	*curr_entry = NULL;
+	int		i,
+			mod,
+			curr_buf_id,
+			new_curr_buf_id,
+			count = 0;
+
+	for (i = 0; i < nnodes; i++)
+	{
+		cb_tag.rnode = rnode[i];
+
+		for (mod = 0; mod < NUM_MAP_PARTITIONS_IN_REL; mod++)
+		{
+			cb_tag.modOfBlockNum = mod;
+
+			combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+			CombinedLockAcquireLWLock(combined_lock);
+
+			hash_entry = (CachedBufEnt *)
+				hash_search_with_hash_value(CachedBufHash,
+											(void *) &cb_tag,
+											hashcode,
+											HASH_FIND,
+											NULL);
+
+			if (!hash_entry)
+			{
+				CombinedLockReleaseLWLock(combined_lock);
+				continue;
+			}
+
+			/* When traversing the dlist, start from head */
+			curr_buf_id = hash_entry->head;
+
+			while(curr_buf_id != CACHED_BUF_END_OF_LIST && count < size)
+			{
+				curr_entry = BufDlistEntArray + curr_buf_id;
+				new_curr_buf_id = curr_entry->next;
+				buf_id_array[count] = curr_buf_id;
+				++count;
+				curr_buf_id = new_curr_buf_id; /* Move pointer */
+			}
+
+			CombinedLockReleaseLWLock(combined_lock);
+		}
+	}
+
+	return count;
+}
+
+
+/*
+ * CachedBufTableDelete
+ *		Unlink the buffer ID from the doubly-linked list, then remove
+ *		the hash entry for the given tag if its list is empty.
+ *
+ * This function must hold spin lock.
+ */
+void
+CachedBufTableDelete(RelFileNode *rnode, BlockNumber *blockNum, int buf_id)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	cb_tag.rnode = *rnode;
+	cb_tag.modOfBlockNum = *blockNum % NUM_MAP_PARTITIONS_IN_REL;
+
+	combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+	CombinedLockAcquireSpinLock(combined_lock);
+
+	/* look up hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &cb_tag,
+									hashcode,
+									HASH_FIND,
+									&found);
+
+	if (!found)		/* tag not found, nothing to do */
+	{
+		CombinedLockReleaseSpinLock(combined_lock);
+		return;
+	}
+
+	cb_dlist_delete(hash_entry, buf_id);
+
+	/*
+	 * If there's no more cached elements for the given tag,
+	 * remove the hash entry.
+	 */
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) &cb_tag,
+										hashcode,
+										HASH_REMOVE,
+										NULL);
+
+		if (!hash_entry)
+			elog(ERROR, "cached buffer hash table corrupted");
+	}
+
+	CombinedLockReleaseSpinLock(combined_lock);
+}
+
+/*
+ * inline functions for the doubly-linked list of cached buffers
+ */
+/* Return the dlist entry */
+static inline BufDlistEnt *
+cb_dlist_entry(int buf_id)
+{
+	return BufDlistEntArray + buf_id;
+}
+
+/* Return the next entry */
+static inline BufDlistEnt *
+cb_dlist_next(BufDlistEnt *entry)
+{
+	return entry->next == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->next);
+}
+
+/* Return the prev entry */
+static inline BufDlistEnt *
+cb_dlist_prev(BufDlistEnt *entry)
+{
+	return entry->prev == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->prev);
+}
+
+/* Return if dlist is empty */
+static inline bool
+cb_dlist_is_empty(CachedBufEnt *hash_entry)
+{
+	return hash_entry->head == CACHED_BUF_END_OF_LIST;
+}
+
+/* Push to head of dlist */
+static inline void
+cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*new_entry = cb_dlist_entry(buf_id);
+	new_entry->next = hash_entry->head;
+
+	if (cb_dlist_is_empty(hash_entry))
+		new_entry->prev = buf_id;
+	else
+	{
+		BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+		new_entry->prev = head_entry->prev;
+		cb_dlist_next(new_entry)->prev = buf_id;
+	}
+	hash_entry->head = buf_id;
+}
+
+/* Remove the buffer ID from dlist */
+static inline void
+cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*curr_entry = cb_dlist_entry(buf_id);
+	BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+	BufDlistEnt	*tail_entry = cb_dlist_entry(head_entry->prev);
+	BufDlistEnt	*next_entry = cb_dlist_next(curr_entry);
+	BufDlistEnt	*prev_entry = cb_dlist_prev(curr_entry);
+
+	/* If only one entry is in the list */
+	if (head_entry == tail_entry)
+	{
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+		return;
+	}
+
+	/*
+	 * If there is a next entry, update its prev field.
+	 * Otherwise, current entry is at tail (but not at head),
+	 * so update the new tail.
+	 */
+	if (next_entry != NULL)
+		next_entry->prev = curr_entry->prev;
+	else
+		head_entry->prev = curr_entry->prev;
+
+	/*
+	 * If the previous entry is not the tail entry, update its
+	 * next field. Otherwise, current entry is at head (but not
+	 * at tail). Update the new head entry and its tail pointer.
+	 */
+	if (prev_entry != tail_entry)
+		prev_entry->next = curr_entry->next;
+	else
+		hash_entry->head = curr_entry->next;
+}
+
+/* Append head of temporary dlist to main dlist */
+static inline void
+cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp)
+{
+	BufDlistEnt	*main_head_entry = cb_dlist_entry(main->head);
+	BufDlistEnt	*temp_head_entry = cb_dlist_entry(temp->head);
+	int	main_tail = main_head_entry->prev;
+	int	temp_tail = temp_head_entry->prev;
+	BufDlistEnt	*main_tail_entry = cb_dlist_entry(main_tail);
+
+	/* Append the temporary dlist to main dlist */
+	main_tail_entry->next = temp->head;
+	temp_head_entry->prev = main_tail;
+	main_head_entry->prev = temp_tail;
+
+	/* Clear the head of temporary dlist */
+	temp->head = CACHED_BUF_END_OF_LIST;
+}
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index bf3b8ad..2bbbcd9 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -337,4 +337,20 @@ extern void DropRelFileNodeLocalBuffers(RelFileNode rnode, ForkNumber forkNum,
 extern void DropRelFileNodeAllLocalBuffers(RelFileNode rnode);
 extern void AtEOXact_LocalBuffers(bool isCommit);
 
+/* in cached_buf.c */
+/* size of array of linked buffer ID's of target relation */
+#define BUF_ID_ARRAY_SIZE	100
+
+extern Size CachedBufShmemSize(void);
+extern void InitCachedBufTable(int size);
+extern void CachedBufTableInsert(RelFileNode *rnode, BlockNumber *blockNum,
+								 int buf_id);
+extern int CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				  int *forknum_indexes, BlockNumber *firstDelBlock,
+				  int *buf_id_array, int size);
+extern int CachedBufLookupAll(RelFileNode *rnode, int nnodes,
+							  int *buf_id_array, int size);
+extern void CachedBufTableDelete(RelFileNode *rnode,  BlockNumber *blockNum,
+								 int buf_id);
+
 #endif							/* BUFMGR_INTERNALS_H */
-- 
1.8.3.1

#12k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#11)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wednesday, March 25, 2020 3:25 PM, Kirk Jamison wrote:

As for the performance and how it affects the read-only workloads.
Using pgbench, I've kept the overload to a minimum, less than 1%.
I'll post follow-up results.

Here's the follow-up results.
I executed the similar tests from top of the thread.
I hope the performance test results shown below would suffice.
If not, I'd appreciate any feedback with regards to test or the patch itself.

A. VACUUM execution + Failover test
- 100GB shared_buffers

1. 1000 tables (18MB)
1.1. Execution Time
- [MASTER] 77755.218 ms (01:17.755)
- [PATCH] Execution Time: 2147.914 ms (00:02.148)
1.2. Failover Time (Recovery WAL Replay):
- [MASTER] 01:37.084 (1 min 37.884 s)
- [PATCH] 1627 ms (1.627 s)

2. 10000 tables (110MB)
2.1. Execution Time
- [MASTER] 844174.572 ms (14:04.175) ~14 min 4.175 s
- [PATCH] 75678.559 ms (01:15.679) ~1 min 15.679 s

2.2. Failover Time:
- [MASTER] est. 14 min++
(I didn't measure anymore because recovery takes
as much as the execution time.)
- [PATCH] 01:25.559 (1 min 25.559 s)

Significant performance results for VACUUM.

B. TPS Regression for READ-ONLY workload
(PREPARED QUERY MODE, NO VACUUM)

# [16 Clients]
- pgbench -n -S -j 16 -c 16 -M prepared -T 60 cachetest

|shbuf |Master |Patch |% reg |
|----------|--------------|---------------|----------|
|128MB| 77,416.76 | 77,162.78 |0.33% |
|1GB | 81,941.30 | 81,812.05 |0.16% |
|2GB | 84,273.69 | 84,356.38 |-0.10%|
|100GB| 83,807.30 | 83,924.68 |-0.14%|

# [1 Client]
- pgbench -n -S -c 1 -M prepared -T 60 cachetest

|shbuf |Master |Patch |% reg |
|----------|--------------|---------------|----------|
|128MB| 12,044.54 | 12,037.13 |0.06% |
|1GB | 12,736.57 | 12,774.77 |-0.30%|
|2GB | 12,948.98 | 13,159.90 |-1.63%|
|100GB| 12,982.98 | 13,064.04 |-0.62%|

Both were run for 10 times and average tps and % regression are
shown above. At some point only minimal overload was caused by
the patch. As for other cases, it has higher tps compared to master.

If it does not make it this CF, I hope to receive feedback in the future
on how to proceed. Thanks in advance!

Regards,
Kirk Jamison

#13k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#12)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi,

Since the last posted version of the patch fails, attached is a rebased version.
Written upthread were performance results and some benefits and challenges.
I'd appreciate your feedback/comments.

Regards,
Kirk Jamison

Attachments:

v8-Optimize-dropping-of-relation-buffers-using-dlist.patchapplication/octet-stream; name=v8-Optimize-dropping-of-relation-buffers-using-dlist.patchDownload
From cff179afa2705ea8e4d2f8296f5166d7267dbfa3 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 5 Jun 2020 01:02:40 +0000
Subject: [PATCH] Optimize dropping of relation buffers using dlist

The invalidation of relation buffers does not depend
on the size of the shared_buffers anymore. This
improves the buffer invalidation process of VACUUM and
TRUNCATE.

Cached buffer hash table is a new auxiliary hash table
structure added to the shared memory. Each hash table
entry contains key (relation file node and modulo of
block number) and buffer ID which points to the head
entry of a doubly-linked list. The cached buffer list
links the to-be-invalidated buffers of a relation to a
doubly-linked list. This significantly improves the
performance of invalidating relation buffers in
DropRelFileNodeBuffers and DropRelFileNodeBuffersAll,
by simply scanning the list rather than scanning the
entire shared_buffers.
---
 src/backend/storage/buffer/Makefile     |   1 +
 src/backend/storage/buffer/buf_init.c   |   6 +
 src/backend/storage/buffer/bufmgr.c     | 163 ++++----
 src/backend/storage/buffer/cached_buf.c | 638 ++++++++++++++++++++++++++++++++
 src/include/storage/buf_internals.h     |  16 +
 5 files changed, 749 insertions(+), 75 deletions(-)
 create mode 100644 src/backend/storage/buffer/cached_buf.c

diff --git a/src/backend/storage/buffer/Makefile b/src/backend/storage/buffer/Makefile
index fd7c40d..3662349 100644
--- a/src/backend/storage/buffer/Makefile
+++ b/src/backend/storage/buffer/Makefile
@@ -16,6 +16,7 @@ OBJS = \
 	buf_init.o \
 	buf_table.o \
 	bufmgr.o \
+	cached_buf.o \
 	freelist.o \
 	localbuf.o
 
diff --git a/src/backend/storage/buffer/buf_init.c b/src/backend/storage/buffer/buf_init.c
index a8ce660..f248fbc 100644
--- a/src/backend/storage/buffer/buf_init.c
+++ b/src/backend/storage/buffer/buf_init.c
@@ -142,6 +142,9 @@ InitBufferPool(void)
 	/* Init other shared buffer-management stuff */
 	StrategyInitialize(!foundDescs);
 
+	/* Init cached buffer hash table and related shmem data structures */
+	InitCachedBufTable(NBuffers);
+
 	/* Initialize per-backend file flush context */
 	WritebackContextInit(&BackendWritebackContext,
 						 &backend_flush_after);
@@ -185,5 +188,8 @@ BufferShmemSize(void)
 	/* size of checkpoint sort array in bufmgr.c */
 	size = add_size(size, mul_size(NBuffers, sizeof(CkptSortItem)));
 
+	/* size of cached buffer shmem data structures */
+	size = add_size(size, CachedBufShmemSize());
+
 	return size;
 }
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 29c9208..67f38d0 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -1329,10 +1329,12 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	if (oldPartitionLock != NULL)
 	{
+		CachedBufTableDelete(&oldTag.rnode, &oldTag.blockNum, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
 		if (oldPartitionLock != newPartitionLock)
 			LWLockRelease(oldPartitionLock);
 	}
+	CachedBufTableInsert(&newTag.rnode, &newTag.blockNum, buf->buf_id);
 
 	LWLockRelease(newPartitionLock);
 
@@ -1442,7 +1444,10 @@ retry:
 	 * Remove the buffer from the lookup hashtable, if it was in there.
 	 */
 	if (oldFlags & BM_TAG_VALID)
+	{
+		CachedBufTableDelete(&oldTag.rnode, &oldTag.blockNum, buf->buf_id);
 		BufTableDelete(&oldTag, oldHash);
+	}
 
 	/*
 	 * Done with mapping lock.
@@ -2936,71 +2941,71 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
 DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
-	int			i;
-	int			j;
+	int			i,
+				nbufs,
+				buf_id_array[BUF_ID_ARRAY_SIZE],
+				forknum_indexes[BUF_ID_ARRAY_SIZE];
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
 	{
 		if (rnode.backend == MyBackendId)
 		{
-			for (j = 0; j < nforks; j++)
-				DropRelFileNodeLocalBuffers(rnode.node, forkNum[j],
-											firstDelBlock[j]);
+			for (i = 0; i < nforks; i++)
+				DropRelFileNodeLocalBuffers(rnode.node, forkNum[i],
+											firstDelBlock[i]);
 		}
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	do
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		nbufs = CachedBufLookup(rnode.node, forkNum, nforks,
+								forknum_indexes, firstDelBlock,
+								buf_id_array, lengthof(buf_id_array));
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+		for (i = 0; i < nbufs; i++)
+		{
+			BufferDesc	*bufHdr = GetBufferDescriptor(buf_id_array[i]);
+			uint32		buf_state;
+			int		index = forknum_indexes[i];
 
-		buf_state = LockBufHdr(bufHdr);
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag
+			 * before we attempt to lock the buffer; this saves a lot of
+			 * lock acquisitions in typical cases.  It should be safe
+			 * because the caller must have AccessExclusiveLock on the
+			 * relation, or some other reason to be certain that no one is
+			 * loading new pages of the rel into the buffer pool.
+			 * (Otherwise we might well miss such pages entirely.)
+			 * Therefore, while the tag might be changing while we look at
+			 * it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible,
+			 * and false positives are safe because we'll recheck after
+			 * getting the buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but
+			 * the incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
+
+			buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
-		{
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
-			{
-				InvalidateBuffer(bufHdr);	/* releases spinlock */
-				break;
-			}
+				bufHdr->tag.forkNum == forkNum[index] &&
+				bufHdr->tag.blockNum >= firstDelBlock[index])
+				InvalidateBuffer(bufHdr); /* releases spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
-	}
+	} while (nbufs == lengthof(buf_id_array));
 }
 
 /* ---------------------------------------------------------------------
@@ -3016,7 +3021,9 @@ void
 DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 {
 	int			i,
-				n = 0;
+				n = 0,
+				nbufs,
+				buf_id_array[BUF_ID_ARRAY_SIZE];
 	RelFileNode *nodes;
 	bool		use_bsearch;
 
@@ -3059,47 +3066,53 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	if (use_bsearch)
 		pg_qsort(nodes, n, sizeof(RelFileNode), rnode_comparator);
 
-	for (i = 0; i < NBuffers; i++)
+	do
 	{
-		RelFileNode *rnode = NULL;
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		nbufs = CachedBufLookupAll(nodes, nnodes, buf_id_array,
+								   lengthof(buf_id_array));
 
-		/*
-		 * As in DropRelFileNodeBuffers, an unlocked precheck should be safe
-		 * and saves some cycles.
-		 */
-
-		if (!use_bsearch)
+		for (i = 0; i < nbufs; i++)
 		{
-			int			j;
+			RelFileNode *rnode = NULL;
+			BufferDesc	*bufHdr = GetBufferDescriptor(buf_id_array[i]);
+			uint32		buf_state;
 
-			for (j = 0; j < n; j++)
+			/*
+			* As in DropRelFileNodeBuffers, an unlocked precheck should be safe
+			* and saves some cycles.
+			*/
+
+			if (!use_bsearch)
 			{
-				if (RelFileNodeEquals(bufHdr->tag.rnode, nodes[j]))
+				int			j;
+
+				for (j = 0; j < n; j++)
 				{
-					rnode = &nodes[j];
-					break;
+					if (RelFileNodeEquals(bufHdr->tag.rnode, nodes[j]))
+					{
+						rnode = &nodes[j];
+						break;
+					}
 				}
 			}
-		}
-		else
-		{
-			rnode = bsearch((const void *) &(bufHdr->tag.rnode),
-							nodes, n, sizeof(RelFileNode),
-							rnode_comparator);
-		}
+			else
+			{
+				rnode = bsearch((const void *) &(bufHdr->tag.rnode),
+								nodes, n, sizeof(RelFileNode),
+								rnode_comparator);
+			}
 
-		/* buffer doesn't belong to any of the given relfilenodes; skip it */
-		if (rnode == NULL)
-			continue;
+			/* buffer doesn't belong to any of the given relfilenodes; skip it */
+			if (rnode == NULL)
+				continue;
 
-		buf_state = LockBufHdr(bufHdr);
-		if (RelFileNodeEquals(bufHdr->tag.rnode, (*rnode)))
-			InvalidateBuffer(bufHdr);	/* releases spinlock */
-		else
-			UnlockBufHdr(bufHdr, buf_state);
-	}
+			buf_state = LockBufHdr(bufHdr);
+			if (RelFileNodeEquals(bufHdr->tag.rnode, (*rnode)))
+				InvalidateBuffer(bufHdr);	/* releases spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);
+		}
+	} while (nbufs == lengthof(buf_id_array));
 
 	pfree(nodes);
 }
diff --git a/src/backend/storage/buffer/cached_buf.c b/src/backend/storage/buffer/cached_buf.c
new file mode 100644
index 0000000..1e38087
--- /dev/null
+++ b/src/backend/storage/buffer/cached_buf.c
@@ -0,0 +1,638 @@
+/*-------------------------------------------------------------------------
+ *
+ * cached_buf.c
+ *	  routines for mapping relations to the indexes of auxillary cached
+ *	  buffers.
+ *
+ * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ *	  src/backend/storage/buffer/cached_buf.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "storage/bufmgr.h"
+#include "storage/buf_internals.h"
+
+
+/*
+ * Each relation and its buffer information are cached in a hash table
+ * located in shared memory. These cached buffers are chained together
+ * in a doubly-linked list.
+ */
+#define NUM_MAP_PARTITIONS_FOR_REL	128	/* relation-level */
+#define NUM_MAP_PARTITIONS_IN_REL	4	/* block-level */
+#define NUM_MAP_PARTITIONS \
+	(NUM_MAP_PARTITIONS_FOR_REL * NUM_MAP_PARTITIONS_IN_REL)
+#define CACHED_BUF_END_OF_LIST	(-1) /* end of doubly linked list */
+
+/* hash table key */
+typedef struct CachedBufTag
+{
+	RelFileNode		rnode;		/* relation */
+	int			modOfBlockNum;	/* modulo of BlockNumber*/
+} CachedBufTag;
+
+/* entry to the cached buffer hash table */
+typedef struct CachedBufEnt
+{
+	CachedBufTag	key;	/* hash table key */
+	int		head;	/* index of dlist */
+} CachedBufEnt;
+
+/* entry to the doubly-linked list */
+typedef struct BufDlistEnt
+{
+	int		prev;
+	int		next;
+} BufDlistEnt;
+
+/* Locks to be used for the hash table operations */
+typedef struct CombinedLock
+{
+	volatile bool	flag;	/* flag for lookup operation */
+	slock_t		spinLock;	/* spinlock to protect the flag */
+	LWLock		lwLock;	/* LWLock used in lookup */
+} CombinedLock;
+
+/* lock for cached buffer hash table */
+typedef struct CachedBufTableLock
+{
+	CombinedLock	cacheLock[NUM_MAP_PARTITIONS];
+	int		cacheTrancheId;
+} CachedBufTableLock;
+
+static HTAB *CachedBufHash = NULL;
+static BufDlistEnt *BufDlistEntArray = NULL;
+static CachedBufTableLock *CachedBufLockArray = NULL;
+static void CombinedLockInitialize(CombinedLock *lock, int trancheId);
+static uint32 CachedBufTableHashCode(CachedBufTag *tag);
+static CombinedLock *GetCachedBufPartitionLock(CachedBufTag *tag,
+											   uint32 *hashcode);
+static void CombinedLockAcquireSpinLock(CombinedLock *lock);
+static void CombinedLockAcquireLWLock(CombinedLock *lock);
+static void CombinedLockReleaseSpinLock(CombinedLock *lock);
+static void CombinedLockReleaseLWLock(CombinedLock *lock);
+static inline BufDlistEnt *cb_dlist_entry(int buf_id);
+static inline BufDlistEnt *cb_dlist_next(BufDlistEnt *entry);
+static inline BufDlistEnt *cb_dlist_prev(BufDlistEnt *entry);
+static inline bool cb_dlist_is_empty(CachedBufEnt *hash_entry);
+static inline void cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id);
+static inline void cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp);
+
+/*
+ * CachedBufShmemSize
+ * 		Estimate space needed for mapping cached buffer hash table
+ *
+ * 		size of lookup table is the desired hash table size
+ *		(possibly more than NBuffers)
+ */
+Size
+CachedBufShmemSize(void)
+{
+	Size		size = 0;
+
+	/* size of cached buffer lookup table */
+	size = add_size(size, hash_estimate_size(NBuffers +
+											 NUM_MAP_PARTITIONS,
+											 sizeof(CachedBufEnt)));
+
+	/* size of cached buffer dlist entry array */
+	size = add_size(size, mul_size(NBuffers, sizeof(BufDlistEnt)));
+
+	/* size of locks */
+	size = add_size(size, mul_size(NBuffers,
+								   sizeof(CachedBufTableLock)));
+
+	return size;
+}
+
+/* Initialize spinlock and LWLock for cached buffer hash table */
+void
+CombinedLockInitialize(CombinedLock *lock, int trancheId)
+{
+	lock->flag = false;
+	SpinLockInit(&lock->spinLock);
+	LWLockInitialize(&lock->lwLock, trancheId);
+}
+
+/*
+ * InitCachedBufTable
+ *      Initialize the cached buffer hash table and related data
+ *      structures at shared memory initialization.
+ */
+void
+InitCachedBufTable(int size)
+{
+	HASHCTL		info;
+	bool		foundList,
+				foundLock;
+	int		i;
+
+	info.keysize = sizeof(CachedBufTag);
+	info.entrysize = sizeof(CachedBufEnt);
+	info.num_partitions = NUM_MAP_PARTITIONS;
+
+	CachedBufHash = ShmemInitHash("Cached Buffer Lookup Table",
+								  size + NUM_MAP_PARTITIONS,
+								  size + NUM_MAP_PARTITIONS,
+								  &info,
+								  HASH_ELEM | HASH_BLOBS |
+								  HASH_PARTITION);
+
+	BufDlistEntArray = ShmemInitStruct("buffer dlist entry array",
+									   size * sizeof(BufDlistEnt),
+									   &foundList);
+
+	CachedBufLockArray = (CachedBufTableLock *)
+			ShmemInitStruct("cached buffer hash partition lock",
+							size * sizeof(CachedBufTableLock),
+							&foundLock);
+
+	if (!foundList && !foundLock)
+	{
+		CachedBufLockArray->cacheTrancheId = LWLockNewTrancheId();
+
+		for (i = 0; i < NUM_MAP_PARTITIONS; i++)
+			CombinedLockInitialize(&CachedBufLockArray->cacheLock[i],
+								   CachedBufLockArray->cacheTrancheId);
+	}
+	LWLockRegisterTranche(CachedBufLockArray->cacheTrancheId,
+						  "cached_buf_tranche_id");
+
+}
+
+ /*
+  * CachedBufTableHashCode
+  *      Compute the hash code associated with tag
+  *
+  * This must be passed to the insert/lookup/delete routines along with the
+  * tag. We do it like this because the callers need to know the hash code
+  * to determine which partition to lock, and we don't want to do the  hash
+  * computation twice (hash_any is a bit slow).
+  */
+static uint32
+CachedBufTableHashCode(CachedBufTag *tag)
+{
+	return get_hash_value(CachedBufHash, (void *) tag);
+}
+
+/*
+ * GetCachedBufPartitionLock
+ * 		Get lock instance for partition of cached buffer lookup table
+ */
+static CombinedLock *
+GetCachedBufPartitionLock(CachedBufTag *tag, uint32 *hashcode)
+{
+	*hashcode = CachedBufTableHashCode(tag);
+
+	return &CachedBufLockArray->cacheLock[*hashcode % NUM_MAP_PARTITIONS];
+}
+
+/*
+ * CombinedLockAcquireSpinLock
+ *		Acquire spinlock to proceed to inserting/deleting hash
+ *		table entries
+ *
+ * This function waits for a lookup process (if flag is true) to finish,
+ * before reacquring a spinlock.
+ */
+void
+CombinedLockAcquireSpinLock(CombinedLock *lock)
+{
+	SpinLockAcquire(&lock->spinLock);
+
+	while (lock->flag)
+	{
+		SpinLockRelease(&lock->spinLock);
+		LWLockAcquire(&lock->lwLock, LW_EXCLUSIVE);
+		LWLockRelease(&lock->lwLock);
+		SpinLockAcquire(&lock->spinLock);
+	}
+}
+
+/*
+ * CombinedLockAcquireLWLock
+ *		Get LWLock instance for partition of cached buffer lookup table
+ *
+ * To prevent other backend processes from modifying the lookup table,
+ * this function sets the CombinedLock's flag to true.
+ */
+void
+CombinedLockAcquireLWLock(CombinedLock *lock)
+{
+	LWLockAcquire(&lock->lwLock, LW_EXCLUSIVE);
+	SpinLockAcquire(&lock->spinLock);
+	lock->flag = true;
+	SpinLockRelease(&lock->spinLock);
+}
+
+/*
+ * CombinedLockReleaseSpinLock
+ *		Release the spinlock used in insert/delete functions
+ */
+void
+CombinedLockReleaseSpinLock(CombinedLock *lock)
+{
+	SpinLockRelease(&lock->spinLock);
+}
+
+/*
+ * CombinedLockReleaseSpinLock
+ *		Release the LWLock used in lookup function after changing the
+ *		CombinedLock's flag to false.
+ */
+void
+CombinedLockReleaseLWLock(CombinedLock *lock)
+{
+	lock->flag = false;
+	LWLockRelease(&lock->lwLock);
+}
+
+/*
+ * CachedBufTableInsert
+ *		Insert a hash table entry for given tag and buffer ID
+ *
+ * If a hash entry for the tag already exists, only buffer ID is inserted.
+ * Chain the buffer ID to the doubly-linked list.
+ *
+ * This function must hold spin lock.
+ */
+void
+CachedBufTableInsert(RelFileNode *rnode, BlockNumber *blockNum, int buf_id)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	cb_tag.rnode = *rnode;
+	cb_tag.modOfBlockNum = *blockNum % NUM_MAP_PARTITIONS_IN_REL;
+
+	combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+	CombinedLockAcquireSpinLock(combined_lock);
+
+	/* look up or create a hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &cb_tag,
+									hashcode,
+									HASH_ENTER,
+									&found);
+
+	/* If not found, initialize linked list */
+	if (!found)
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+
+	cb_dlist_push_head(hash_entry, buf_id);
+
+	CombinedLockReleaseSpinLock(combined_lock);
+}
+
+/*
+ * CachedBufLookup
+ *		Lookup the buffers for the given tag in the cached buffer
+ *		hash table. Insert the buffer ID to the array of buffer
+ *		IDs, and return the total number of buffers to be invalidated.
+ *
+ * This function must hold exclusive LWLock for tag's partition.
+ */
+int
+CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				int *forknum_indexes, BlockNumber *firstDelBlock,
+				int *buf_id_array, int size)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry,
+					temp_hash_entry;
+	BufDlistEnt	*curr_entry = NULL;
+	int		mod,
+			i,
+			curr_buf_id,
+			new_curr_buf_id,
+			count = 0;
+	bool		target_buf = false;
+
+	for (mod = 0; mod < NUM_MAP_PARTITIONS_IN_REL; mod++)
+	{
+		cb_tag.rnode = rnode;
+		cb_tag.modOfBlockNum = mod;
+
+		combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+		CombinedLockAcquireLWLock(combined_lock);
+
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) &cb_tag,
+										hashcode,
+										HASH_FIND,
+										NULL);
+
+		if (!hash_entry)
+		{
+			CombinedLockReleaseLWLock(combined_lock);
+			continue;
+		}
+
+		/* Initial temporary dlist */
+		temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+
+		/* When traversing the main dlist, start from head */
+		curr_buf_id = hash_entry->head;
+
+		while(curr_buf_id != CACHED_BUF_END_OF_LIST && count < size)
+		{
+			BufferDesc *bufHdr = GetBufferDescriptor(curr_buf_id);
+			curr_entry = BufDlistEntArray + curr_buf_id;
+			new_curr_buf_id = curr_entry->next;
+
+			/* Check if it's our target buffer */
+			for (i = 0; i < nforks; i++)
+			{
+				if (bufHdr->tag.forkNum != forkNum[i])
+					continue;
+				else
+				{
+					if (bufHdr->tag.blockNum >= firstDelBlock[i])
+						target_buf = true;
+					break;
+				}
+			}
+
+			if (target_buf)
+			{
+				forknum_indexes[count] = i;
+				buf_id_array[count] = curr_buf_id;
+				++count;
+			}
+			else
+			{
+				/*
+				* It's not the target buffer. Remove the current buffer ID
+				* from the current list of target buffer IDs and store it
+				* to a temporary list.
+				*/
+				cb_dlist_delete(hash_entry, curr_buf_id);
+				cb_dlist_push_head(&temp_hash_entry, curr_buf_id);
+			}
+			/* Move current pointer to next */
+			curr_buf_id = new_curr_buf_id;
+		}
+
+		/* Check if main dlist is now empty */
+		if (cb_dlist_is_empty(hash_entry))
+		{
+			hash_entry->head = temp_hash_entry.head;
+			temp_hash_entry.head = CACHED_BUF_END_OF_LIST;
+		}
+
+		/* If we have a temporary dlist, append it to the main dlist */
+		if (!cb_dlist_is_empty(hash_entry) &&
+			!cb_dlist_is_empty(&temp_hash_entry))
+			cb_dlist_combine(hash_entry, &temp_hash_entry);
+
+		CombinedLockReleaseLWLock(combined_lock);
+	}
+
+	return count;
+}
+
+/*
+ * CachedBufLookupAll
+ *		Lookup all the buffers of specified relations in the cached buffer
+ *		hash table. Store all the buffer IDs to the array, and return
+ *		the total number of buffers to be invalidated.
+ *
+ * This function must hold exclusive LWLock for tag's partition.
+ */
+int
+CachedBufLookupAll(RelFileNode *rnode, int nnodes, int *buf_id_array,
+				   int size)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	BufDlistEnt	*curr_entry = NULL;
+	int		i,
+			mod,
+			curr_buf_id,
+			new_curr_buf_id,
+			count = 0;
+
+	for (i = 0; i < nnodes; i++)
+	{
+		cb_tag.rnode = rnode[i];
+
+		for (mod = 0; mod < NUM_MAP_PARTITIONS_IN_REL; mod++)
+		{
+			cb_tag.modOfBlockNum = mod;
+
+			combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+			CombinedLockAcquireLWLock(combined_lock);
+
+			hash_entry = (CachedBufEnt *)
+				hash_search_with_hash_value(CachedBufHash,
+											(void *) &cb_tag,
+											hashcode,
+											HASH_FIND,
+											NULL);
+
+			if (!hash_entry)
+			{
+				CombinedLockReleaseLWLock(combined_lock);
+				continue;
+			}
+
+			/* When traversing the dlist, start from head */
+			curr_buf_id = hash_entry->head;
+
+			while(curr_buf_id != CACHED_BUF_END_OF_LIST && count < size)
+			{
+				curr_entry = BufDlistEntArray + curr_buf_id;
+				new_curr_buf_id = curr_entry->next;
+				buf_id_array[count] = curr_buf_id;
+				++count;
+				curr_buf_id = new_curr_buf_id; /* Move pointer */
+			}
+
+			CombinedLockReleaseLWLock(combined_lock);
+		}
+	}
+
+	return count;
+}
+
+
+/*
+ * CachedBufTableDelete
+ *		Unlink the buffer ID from the doubly-linked list, then remove
+ *		the hash entry for the given tag if its list is empty.
+ *
+ * This function must hold spin lock.
+ */
+void
+CachedBufTableDelete(RelFileNode *rnode, BlockNumber *blockNum, int buf_id)
+{
+	CachedBufTag	cb_tag;
+	uint32		hashcode;
+	CombinedLock	*combined_lock;
+	CachedBufEnt	*hash_entry;
+	bool		found;
+
+	cb_tag.rnode = *rnode;
+	cb_tag.modOfBlockNum = *blockNum % NUM_MAP_PARTITIONS_IN_REL;
+
+	combined_lock = GetCachedBufPartitionLock(&cb_tag, &hashcode);
+	CombinedLockAcquireSpinLock(combined_lock);
+
+	/* look up hash table entry */
+	hash_entry = (CachedBufEnt *)
+		hash_search_with_hash_value(CachedBufHash,
+									(void *) &cb_tag,
+									hashcode,
+									HASH_FIND,
+									&found);
+
+	if (!found)		/* tag not found, nothing to do */
+	{
+		CombinedLockReleaseSpinLock(combined_lock);
+		return;
+	}
+
+	cb_dlist_delete(hash_entry, buf_id);
+
+	/*
+	 * If there's no more cached elements for the given tag,
+	 * remove the hash entry.
+	 */
+	if (cb_dlist_is_empty(hash_entry))
+	{
+		hash_entry = (CachedBufEnt *)
+			hash_search_with_hash_value(CachedBufHash,
+										(void *) &cb_tag,
+										hashcode,
+										HASH_REMOVE,
+										NULL);
+
+		if (!hash_entry)
+			elog(ERROR, "cached buffer hash table corrupted");
+	}
+
+	CombinedLockReleaseSpinLock(combined_lock);
+}
+
+/*
+ * inline functions for the doubly-linked list of cached buffers
+ */
+/* Return the dlist entry */
+static inline BufDlistEnt *
+cb_dlist_entry(int buf_id)
+{
+	return BufDlistEntArray + buf_id;
+}
+
+/* Return the next entry */
+static inline BufDlistEnt *
+cb_dlist_next(BufDlistEnt *entry)
+{
+	return entry->next == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->next);
+}
+
+/* Return the prev entry */
+static inline BufDlistEnt *
+cb_dlist_prev(BufDlistEnt *entry)
+{
+	return entry->prev == CACHED_BUF_END_OF_LIST ?
+							NULL : cb_dlist_entry(entry->prev);
+}
+
+/* Return if dlist is empty */
+static inline bool
+cb_dlist_is_empty(CachedBufEnt *hash_entry)
+{
+	return hash_entry->head == CACHED_BUF_END_OF_LIST;
+}
+
+/* Push to head of dlist */
+static inline void
+cb_dlist_push_head(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*new_entry = cb_dlist_entry(buf_id);
+	new_entry->next = hash_entry->head;
+
+	if (cb_dlist_is_empty(hash_entry))
+		new_entry->prev = buf_id;
+	else
+	{
+		BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+		new_entry->prev = head_entry->prev;
+		cb_dlist_next(new_entry)->prev = buf_id;
+	}
+	hash_entry->head = buf_id;
+}
+
+/* Remove the buffer ID from dlist */
+static inline void
+cb_dlist_delete(CachedBufEnt *hash_entry, int buf_id)
+{
+	BufDlistEnt	*curr_entry = cb_dlist_entry(buf_id);
+	BufDlistEnt	*head_entry = cb_dlist_entry(hash_entry->head);
+	BufDlistEnt	*tail_entry = cb_dlist_entry(head_entry->prev);
+	BufDlistEnt	*next_entry = cb_dlist_next(curr_entry);
+	BufDlistEnt	*prev_entry = cb_dlist_prev(curr_entry);
+
+	/* If only one entry is in the list */
+	if (head_entry == tail_entry)
+	{
+		hash_entry->head = CACHED_BUF_END_OF_LIST;
+		return;
+	}
+
+	/*
+	 * If there is a next entry, update its prev field.
+	 * Otherwise, current entry is at tail (but not at head),
+	 * so update the new tail.
+	 */
+	if (next_entry != NULL)
+		next_entry->prev = curr_entry->prev;
+	else
+		head_entry->prev = curr_entry->prev;
+
+	/*
+	 * If the previous entry is not the tail entry, update its
+	 * next field. Otherwise, current entry is at head (but not
+	 * at tail). Update the new head entry and its tail pointer.
+	 */
+	if (prev_entry != tail_entry)
+		prev_entry->next = curr_entry->next;
+	else
+		hash_entry->head = curr_entry->next;
+}
+
+/* Append head of temporary dlist to main dlist */
+static inline void
+cb_dlist_combine(CachedBufEnt *main, CachedBufEnt *temp)
+{
+	BufDlistEnt	*main_head_entry = cb_dlist_entry(main->head);
+	BufDlistEnt	*temp_head_entry = cb_dlist_entry(temp->head);
+	int	main_tail = main_head_entry->prev;
+	int	temp_tail = temp_head_entry->prev;
+	BufDlistEnt	*main_tail_entry = cb_dlist_entry(main_tail);
+
+	/* Append the temporary dlist to main dlist */
+	main_tail_entry->next = temp->head;
+	temp_head_entry->prev = main_tail;
+	main_head_entry->prev = temp_tail;
+
+	/* Clear the head of temporary dlist */
+	temp->head = CACHED_BUF_END_OF_LIST;
+}
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index e57f84e..1d9bf54 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -338,4 +338,20 @@ extern void DropRelFileNodeLocalBuffers(RelFileNode rnode, ForkNumber forkNum,
 extern void DropRelFileNodeAllLocalBuffers(RelFileNode rnode);
 extern void AtEOXact_LocalBuffers(bool isCommit);
 
+/* in cached_buf.c */
+/* size of array of linked buffer ID's of target relation */
+#define BUF_ID_ARRAY_SIZE	100
+
+extern Size CachedBufShmemSize(void);
+extern void InitCachedBufTable(int size);
+extern void CachedBufTableInsert(RelFileNode *rnode, BlockNumber *blockNum,
+								 int buf_id);
+extern int CachedBufLookup(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+				  int *forknum_indexes, BlockNumber *firstDelBlock,
+				  int *buf_id_array, int size);
+extern int CachedBufLookupAll(RelFileNode *rnode, int nnodes,
+							  int *buf_id_array, int size);
+extern void CachedBufTableDelete(RelFileNode *rnode,  BlockNumber *blockNum,
+								 int buf_id);
+
 #endif							/* BUFMGR_INTERNALS_H */
-- 
1.8.3.1

#14Konstantin Knizhnik
k.knizhnik@postgrespro.ru
In reply to: k.jamison@fujitsu.com (#13)
Re: [Patch] Optimize dropping of relation buffers using dlist

On 17.06.2020 09:14, k.jamison@fujitsu.com wrote:

Hi,

Since the last posted version of the patch fails, attached is a rebased version.
Written upthread were performance results and some benefits and challenges.
I'd appreciate your feedback/comments.

Regards,
Kirk Jamison

As far as i understand this patch can provide significant improvement of
performance only in case of
recovery  of truncates of large number of tables. You have added shared
hash of relation buffers and certainly if adds some
extra overhead. According to your latest results this overhead is quite
small. But it will be hard to prove that there will be no noticeable
regression
at some workloads.

I wonder if you have considered case of local hash (maintained only
during recovery)?
If there is after-crash recovery, then there will be no concurrent
access to shared buffers and this hash will be up-to-date.
in case of hot-standby replica we can use some simple invalidation (just
one flag or counter which indicates that buffer cache was updated).
This hash also can be constructed on demand when DropRelFileNodeBuffers
is called first time (so w have to scan all buffers once, but subsequent
drop operation will be fast.

i have not thought much about it, but it seems to me that as far as this
problem only affects recovery, we do not need shared hash for it.

#15k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Konstantin Knizhnik (#14)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wednesday, July 29, 2020 4:55 PM, Konstantin Knizhnik wrote:

On 17.06.2020 09:14, k.jamison@fujitsu.com wrote:

Hi,

Since the last posted version of the patch fails, attached is a rebased version.
Written upthread were performance results and some benefits and challenges.
I'd appreciate your feedback/comments.

Regards,
Kirk Jamison

As far as i understand this patch can provide significant improvement of
performance only in case of recovery  of truncates of large number of tables. You
have added shared hash of relation buffers and certainly if adds some extra
overhead. According to your latest results this overhead is quite small. But it will
be hard to prove that there will be no noticeable regression at some workloads.

Thank you for taking a look at this.

Yes, one of the aims is to speed up recovery of truncations, but at the same time the
patch also improves autovacuum, vacuum and relation truncate index executions.
I showed results of pgbench results above for different types of workloads,
but I am not sure if those are validating enough...

I wonder if you have considered case of local hash (maintained only during
recovery)?
If there is after-crash recovery, then there will be no concurrent access to shared
buffers and this hash will be up-to-date.
in case of hot-standby replica we can use some simple invalidation (just one flag
or counter which indicates that buffer cache was updated).
This hash also can be constructed on demand when DropRelFileNodeBuffers is
called first time (so w have to scan all buffers once, but subsequent drop
operation will be fast.

i have not thought much about it, but it seems to me that as far as this problem
only affects recovery, we do not need shared hash for it.

The idea of the patch is to mark the relation buffers to be dropped after scanning
the whole shared buffers, and store them into shared memory maintained in a dlist,
and traverse the dlist on the next scan.
But I understand the point that it is expensive and may cause overhead, that is why
I tried to define a macro to limit the number of pages that we can cache for cases
that lookup cost can be problematic (i.e. too many pages of relation).

#define BUF_ID_ARRAY_SIZE 100
int buf_id_array[BUF_ID_ARRAY_SIZE];
int forknum_indexes[BUF_ID_ARRAY_SIZE];

In DropRelFileNodeBuffers
do
{
nbufs = CachedBlockLookup(..., forknum_indexes, buf_id_array, lengthof(buf_id_array));
for (i = 0; i < nbufs; i++)
{
...
}
} while (nbufs == lengthof(buf_id_array));

Perhaps the patch affects complexities so we want to keep it simpler, or commit piece by piece?
I will look further into your suggestion of maintaining local hash only during recovery.
Thank you for the suggestion.

Regards,
Kirk Jamison

#16Konstantin Knizhnik
knizhnik@garret.ru
In reply to: k.jamison@fujitsu.com (#15)
Re: [Patch] Optimize dropping of relation buffers using dlist

The following review has been posted through the commitfest application:
make installcheck-world: tested, passed
Implements feature: tested, passed
Spec compliant: not tested
Documentation: not tested

I have tested this patch at various workloads and hardware (including Power2 server with 384 virtual cores)
and didn't find performance regression.

The new status of this patch is: Ready for Committer

#17k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Konstantin Knizhnik (#16)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Friday, July 31, 2020 2:37 AM, Konstantin Knizhnik wrote:

The following review has been posted through the commitfest application:
make installcheck-world: tested, passed
Implements feature: tested, passed
Spec compliant: not tested
Documentation: not tested

I have tested this patch at various workloads and hardware (including Power2
server with 384 virtual cores) and didn't find performance regression.

The new status of this patch is: Ready for Committer

Thank you very much, Konstantin, for testing the patch for different workloads.
I wonder if I need to modify some documentations.
I'll leave the final review to the committer/s as well.

Regards,
Kirk Jamison

#18Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#10)
Re: [Patch] Optimize dropping of relation buffers using dlist

Robert Haas <robertmhaas@gmail.com> writes:

Unfortunately, I don't have time for detailed review of this. I am
suspicious that there are substantial performance regressions that you
just haven't found yet. I would not take the position that this is a
completely hopeless approach, or anything like that, but neither would
I conclude that the tests shown so far are anywhere near enough to be
confident that there are no problems.

I took a quick look through the v8 patch, since it's marked RFC, and
my feeling is about the same as Robert's: it is just about impossible
to believe that doubling (or more) the amount of hashtable manipulation
involved in allocating a buffer won't hurt common workloads. The
offered pgbench results don't reassure me; we've so often found that
pgbench fails to expose performance problems, except maybe when it's
used just so.

But aside from that, I noted a number of things I didn't like a bit:

* The amount of new shared memory this needs seems several orders
of magnitude higher than what I'd call acceptable: according to my
measurements it's over 10KB per shared buffer! Most of that is going
into the CachedBufTableLock data structure, which seems fundamentally
misdesigned --- how could we be needing a lock per map partition *per
buffer*? For comparison, the space used by buf_table.c is about 56
bytes per shared buffer; I think this needs to stay at least within
hailing distance of there.

* It is fairly suspicious that the new data structure is manipulated
while holding per-partition locks for the existing buffer hashtable.
At best that seems bad for concurrency, and at worst it could result
in deadlocks, because I doubt we can assume that the new hash table
has partition boundaries identical to the old one.

* More generally, it seems like really poor design that this has been
written completely independently of the existing buffer hash table.
Can't we get any benefit by merging them somehow?

* I do not like much of anything in the code details. "CachedBuf"
is as unhelpful as could be as a data structure identifier --- what
exactly is not "cached" about shared buffers already? "CombinedLock"
is not too helpful either, nor could I find any documentation explaining
why you need to invent new locking technology in the first place.
At best, CombinedLockAcquireSpinLock seems like a brute-force approach
to an undocumented problem.

* The commentary overall is far too sparse to be of any value ---
basically, any reader will have to reverse-engineer your entire design.
That's not how we do things around here. There should be either a README,
or a long file header comment, explaining what's going on, how the data
structure is organized, and what the locking requirements are.
See src/backend/storage/buffer/README for the sort of documentation
that I think this needs.

Even if I were convinced that there's no performance gotchas,
I wouldn't commit this in anything like its current form.

Robert again:

Also, systems with very large shared_buffers settings are becoming
more common, and probably will continue to become more common, so I
don't think we can dismiss that as an edge case any more. People don't
want to run with an 8GB cache on a 1TB server.

I do agree that it'd be great to improve this area. Just not convinced
that this is how.

regards, tom lane

#19Andres Freund
andres@anarazel.de
In reply to: Tom Lane (#18)
Re: [Patch] Optimize dropping of relation buffers using dlist

Hi,

On 2020-07-31 13:39:37 -0400, Tom Lane wrote:

Robert Haas <robertmhaas@gmail.com> writes:

Unfortunately, I don't have time for detailed review of this. I am
suspicious that there are substantial performance regressions that you
just haven't found yet. I would not take the position that this is a
completely hopeless approach, or anything like that, but neither would
I conclude that the tests shown so far are anywhere near enough to be
confident that there are no problems.

I took a quick look through the v8 patch, since it's marked RFC, and
my feeling is about the same as Robert's: it is just about impossible
to believe that doubling (or more) the amount of hashtable manipulation
involved in allocating a buffer won't hurt common workloads. The
offered pgbench results don't reassure me; we've so often found that
pgbench fails to expose performance problems, except maybe when it's
used just so.

Indeed. The buffer mapping hashtable already is visible as a major
bottleneck in a number of workloads. Even in readonly pgbench if s_b is
large enough (so the hashtable is larger than the cache). Not to speak
of things like a cached sequential scan with a cheap qual and wide rows.

Robert again:

Also, systems with very large shared_buffers settings are becoming
more common, and probably will continue to become more common, so I
don't think we can dismiss that as an edge case any more. People don't
want to run with an 8GB cache on a 1TB server.

I do agree that it'd be great to improve this area. Just not convinced
that this is how.

Wonder if the temporary fix is just to do explicit hashtable probes for
all pages iff the size of the relation is < s_b / 500 or so. That'll
address the case where small tables are frequently dropped - and
dropping large relations is more expensive from the OS and data loading
perspective, so it's not gonna happen as often.

Greetings,

Andres Freund

#20Tom Lane
tgl@sss.pgh.pa.us
In reply to: Andres Freund (#19)
Re: [Patch] Optimize dropping of relation buffers using dlist

Andres Freund <andres@anarazel.de> writes:

Indeed. The buffer mapping hashtable already is visible as a major
bottleneck in a number of workloads. Even in readonly pgbench if s_b is
large enough (so the hashtable is larger than the cache). Not to speak
of things like a cached sequential scan with a cheap qual and wide rows.

To be fair, the added overhead is in buffer allocation not buffer lookup.
So it shouldn't add cost to fully-cached cases. As Tomas noted upthread,
the potential trouble spot is where the working set is bigger than shared
buffers but still fits in RAM (so there's no actual I/O needed, but we do
still have to shuffle buffers a lot).

Wonder if the temporary fix is just to do explicit hashtable probes for
all pages iff the size of the relation is < s_b / 500 or so. That'll
address the case where small tables are frequently dropped - and
dropping large relations is more expensive from the OS and data loading
perspective, so it's not gonna happen as often.

Oooh, interesting idea. We'd need a reliable idea of how long the
relation had been (preferably without adding an lseek call), but maybe
that's do-able.

regards, tom lane

#21Andres Freund
andres@anarazel.de
In reply to: Tom Lane (#20)
Re: [Patch] Optimize dropping of relation buffers using dlist

Hi,

On 2020-07-31 15:50:04 -0400, Tom Lane wrote:

Andres Freund <andres@anarazel.de> writes:

Indeed. The buffer mapping hashtable already is visible as a major
bottleneck in a number of workloads. Even in readonly pgbench if s_b is
large enough (so the hashtable is larger than the cache). Not to speak
of things like a cached sequential scan with a cheap qual and wide rows.

To be fair, the added overhead is in buffer allocation not buffer lookup.
So it shouldn't add cost to fully-cached cases. As Tomas noted upthread,
the potential trouble spot is where the working set is bigger than shared
buffers but still fits in RAM (so there's no actual I/O needed, but we do
still have to shuffle buffers a lot).

Oh, right, not sure what I was thinking.

Wonder if the temporary fix is just to do explicit hashtable probes for
all pages iff the size of the relation is < s_b / 500 or so. That'll
address the case where small tables are frequently dropped - and
dropping large relations is more expensive from the OS and data loading
perspective, so it's not gonna happen as often.

Oooh, interesting idea. We'd need a reliable idea of how long the
relation had been (preferably without adding an lseek call), but maybe
that's do-able.

IIRC we already do smgrnblocks nearby, when doing the truncation (to
figure out which segments we need to remove). Perhaps we can arrange to
combine the two? The layering probably makes that somewhat ugly :(

We could also just use pg_class.relpages. It'll probably mostly be
accurate enough?

Or we could just cache the result of the last smgrnblocks call...

One of the cases where this type of strategy is most intersting to me is
the partial truncations that autovacuum does... There we even know the
range of tables ahead of time.

Greetings,

Andres Freund

#22k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Andres Freund (#21)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Saturday, August 1, 2020 5:24 AM, Andres Freund wrote:

Hi,
Thank you for your constructive review and comments.
Sorry for the late reply.

Hi,

On 2020-07-31 15:50:04 -0400, Tom Lane wrote:

Andres Freund <andres@anarazel.de> writes:

Indeed. The buffer mapping hashtable already is visible as a major
bottleneck in a number of workloads. Even in readonly pgbench if s_b
is large enough (so the hashtable is larger than the cache). Not to
speak of things like a cached sequential scan with a cheap qual and wide

rows.

To be fair, the added overhead is in buffer allocation not buffer lookup.
So it shouldn't add cost to fully-cached cases. As Tomas noted
upthread, the potential trouble spot is where the working set is
bigger than shared buffers but still fits in RAM (so there's no actual
I/O needed, but we do still have to shuffle buffers a lot).

Oh, right, not sure what I was thinking.

Wonder if the temporary fix is just to do explicit hashtable probes
for all pages iff the size of the relation is < s_b / 500 or so.
That'll address the case where small tables are frequently dropped -
and dropping large relations is more expensive from the OS and data
loading perspective, so it's not gonna happen as often.

Oooh, interesting idea. We'd need a reliable idea of how long the
relation had been (preferably without adding an lseek call), but maybe
that's do-able.

IIRC we already do smgrnblocks nearby, when doing the truncation (to figure out
which segments we need to remove). Perhaps we can arrange to combine the
two? The layering probably makes that somewhat ugly :(

We could also just use pg_class.relpages. It'll probably mostly be accurate
enough?

Or we could just cache the result of the last smgrnblocks call...

One of the cases where this type of strategy is most intersting to me is the partial
truncations that autovacuum does... There we even know the range of tables
ahead of time.

Konstantin tested it on various workloads and saw no regression.
But I understand the sentiment on the added overhead on BufferAlloc.
Regarding the case where the patch would potentially affect workloads that fit into
RAM but not into shared buffers, could one of Andres' suggested idea/s above address
that, in addition to this patch's possible shared invalidation fix? Could that settle
the added overhead in BufferAlloc() as temporary fix?
Thomas Munro is also working on caching relation sizes [1]/messages/by-id/CA+hUKGKEW7-9pq+s2_4Q-Fcpr9cc7_0b3pkno5qzPKC3y2nOPA@mail.gmail.com, maybe that way we
could get the latest known relation size. Currently, it's possible only during
recovery in smgrnblocks.

Tom Lane wrote:

But aside from that, I noted a number of things I didn't like a bit:

* The amount of new shared memory this needs seems several orders of
magnitude higher than what I'd call acceptable: according to my measurements
it's over 10KB per shared buffer! Most of that is going into the
CachedBufTableLock data structure, which seems fundamentally misdesigned ---
how could we be needing a lock per map partition *per buffer*? For comparison,
the space used by buf_table.c is about 56 bytes per shared buffer; I think this
needs to stay at least within hailing distance of there.

* It is fairly suspicious that the new data structure is manipulated while holding
per-partition locks for the existing buffer hashtable.
At best that seems bad for concurrency, and at worst it could result in deadlocks,
because I doubt we can assume that the new hash table has partition boundaries
identical to the old one.

* More generally, it seems like really poor design that this has been written
completely independently of the existing buffer hash table.
Can't we get any benefit by merging them somehow?

The original aim is to just shorten the recovery process, and eventually the speedup
on both vacuum and truncate process are just added bonus.
Given that we don't have a shared invalidation mechanism in place yet like radix tree
buffer mapping which is complex, I thought a patch like mine could be an alternative
approach to that. So I want to improve the patch further.
I hope you can help me clarify the direction, so that I can avoid going farther away
from what the community wants.
1. Both normal operations and recovery process
2. Improve recovery process only

For 1, the current patch aims to touch on that, but further design improvement is needed.
It would be ideal to modify the BufferDesc, but that cannot be expanded anymore because
it would exceed the CPU cache line size. So I added new data structures (hash table,
dlist, lock) instead of modifying the existing ones.
The new hash table ensures that it's identical to the old one with the use of the same
Relfilenode in the key and a lock when inserting and deleting buffers from buffer table,
as well as during lookups. As for the partition locking, I added it to reduce lock contention.
Tomas Vondra reported regression and mainly its due to buffer mapping locks in V4 and
previous patch versions. So from V5, I used spinlock when inserting/deleting buffers,
to prevent modification when concurrent lookup is happening. LWLock is acquired when
we're doing lookup operation.
If we want this direction, I hope to address Tom's comments in the next patch version.
I admit that this patch needs reworking on shmem resource consumption and clarifying
the design/approach more, i.e. how it affects the existing buffer allocation and
invalidation process, lock mechanism, etc.

If we're going for 2, Konstantin suggested an idea in the previous email:

I wonder if you have considered case of local hash (maintained only during recovery)?
If there is after-crash recovery, then there will be no concurrent
access to shared buffers and this hash will be up-to-date.
in case of hot-standby replica we can use some simple invalidation (just
one flag or counter which indicates that buffer cache was updated).
This hash also can be constructed on demand when DropRelFileNodeBuffers
is called first time (so w have to scan all buffers once, but subsequent
drop operation will be fast.

I'm examining this, but I am not sure if I got the correct understanding. Please correct
me if I'm wrong.
I think above is a suggestion wherein the postgres startup process uses local hash table
to keep track of the buffers of relations. Since there may be other read-only sessions which
read from disk, evict cached blocks, and modify the shared_buffers, the flag would be updated.
We could do it during recovery, then release it as recovery completes.

I haven't looked deeply yet into the source code but we maybe can modify the REDO
(main redo do-while loop) in StartupXLOG() once the read-only connections are consistent.
It would also be beneficial to construct this local hash when DropRefFileNodeBuffers()
is called for the first time, so the whole share buffers is scanned initially, then as
you mentioned subsequent dropping will be fast. (similar behavior to what the patch does)

Do you think this is feasible to be implemented? Or should we explore another approach?

I'd really appreciate your ideas, feedback, suggestions, and advice.
Thank you again for the review.

Regards
Kirk Jamison

[1]: /messages/by-id/CA+hUKGKEW7-9pq+s2_4Q-Fcpr9cc7_0b3pkno5qzPKC3y2nOPA@mail.gmail.com

#23Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: k.jamison@fujitsu.com (#22)
3 attachment(s)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Aug 06, 2020 at 01:23:31AM +0000, k.jamison@fujitsu.com wrote:

On Saturday, August 1, 2020 5:24 AM, Andres Freund wrote:

Hi,
Thank you for your constructive review and comments.
Sorry for the late reply.

Hi,

On 2020-07-31 15:50:04 -0400, Tom Lane wrote:

Andres Freund <andres@anarazel.de> writes:

Indeed. The buffer mapping hashtable already is visible as a major
bottleneck in a number of workloads. Even in readonly pgbench if s_b
is large enough (so the hashtable is larger than the cache). Not to
speak of things like a cached sequential scan with a cheap qual and wide

rows.

To be fair, the added overhead is in buffer allocation not buffer lookup.
So it shouldn't add cost to fully-cached cases. As Tomas noted
upthread, the potential trouble spot is where the working set is
bigger than shared buffers but still fits in RAM (so there's no actual
I/O needed, but we do still have to shuffle buffers a lot).

Oh, right, not sure what I was thinking.

Wonder if the temporary fix is just to do explicit hashtable probes
for all pages iff the size of the relation is < s_b / 500 or so.
That'll address the case where small tables are frequently dropped -
and dropping large relations is more expensive from the OS and data
loading perspective, so it's not gonna happen as often.

Oooh, interesting idea. We'd need a reliable idea of how long the
relation had been (preferably without adding an lseek call), but maybe
that's do-able.

IIRC we already do smgrnblocks nearby, when doing the truncation (to figure out
which segments we need to remove). Perhaps we can arrange to combine the
two? The layering probably makes that somewhat ugly :(

We could also just use pg_class.relpages. It'll probably mostly be accurate
enough?

Or we could just cache the result of the last smgrnblocks call...

One of the cases where this type of strategy is most intersting to me is the partial
truncations that autovacuum does... There we even know the range of tables
ahead of time.

Konstantin tested it on various workloads and saw no regression.

Unfortunately Konstantin did not share any details about what workloads
he tested, what config etc. But I find the "no regression" hypothesis
rather hard to believe, because we're adding non-trivial amount of code
to a place that can be quite hot.

And I can trivially reproduce measurable (and significant) regression
using a very simple pgbench read-only test, with amount of data that
exceeds shared buffers but fits into RAM.

The following numbers are from a x86_64 machine with 16 cores (32 w HT),
64GB of RAM, and 8GB shared buffers, using pgbench scale 1000 (so 16GB,
i.e. twice the SB size).

With simple "pgbench -S" tests (warmup and then 15 x 1-minute runs with
1, 8 and 16 clients - see the attached script for details) I see this:

1 client 8 clients 16 clients
----------------------------------------------
master 38249 236336 368591
patched 35853 217259 349248
-6% -8% -5%

This is average of the runs, but the conclusions for medians are almost
exactly te same.

But I understand the sentiment on the added overhead on BufferAlloc.
Regarding the case where the patch would potentially affect workloads
that fit into RAM but not into shared buffers, could one of Andres'
suggested idea/s above address that, in addition to this patch's
possible shared invalidation fix? Could that settle the added overhead
in BufferAlloc() as temporary fix?

Not sure.

Thomas Munro is also working on caching relation sizes [1], maybe that
way we could get the latest known relation size. Currently, it's
possible only during recovery in smgrnblocks.

It's not clear to me how would knowing the relation size help reducing
the overhead of this patch?

Can't we somehow identify cases when this optimization might help and
only actually enable it in those cases? Like in a recovery, with a lot
of truncates, or something like that.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

run.shapplication/x-shDownload
master.csvtext/csv; charset=us-asciiDownload
patched.csvtext/csv; charset=us-asciiDownload
#24Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#22)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Aug 6, 2020 at 6:53 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Saturday, August 1, 2020 5:24 AM, Andres Freund wrote:

Hi,
Thank you for your constructive review and comments.
Sorry for the late reply.

Hi,

On 2020-07-31 15:50:04 -0400, Tom Lane wrote:

Andres Freund <andres@anarazel.de> writes:

Indeed. The buffer mapping hashtable already is visible as a major
bottleneck in a number of workloads. Even in readonly pgbench if s_b
is large enough (so the hashtable is larger than the cache). Not to
speak of things like a cached sequential scan with a cheap qual and wide

rows.

To be fair, the added overhead is in buffer allocation not buffer lookup.
So it shouldn't add cost to fully-cached cases. As Tomas noted
upthread, the potential trouble spot is where the working set is
bigger than shared buffers but still fits in RAM (so there's no actual
I/O needed, but we do still have to shuffle buffers a lot).

Oh, right, not sure what I was thinking.

Wonder if the temporary fix is just to do explicit hashtable probes
for all pages iff the size of the relation is < s_b / 500 or so.
That'll address the case where small tables are frequently dropped -
and dropping large relations is more expensive from the OS and data
loading perspective, so it's not gonna happen as often.

Oooh, interesting idea. We'd need a reliable idea of how long the
relation had been (preferably without adding an lseek call), but maybe
that's do-able.

IIRC we already do smgrnblocks nearby, when doing the truncation (to figure out
which segments we need to remove). Perhaps we can arrange to combine the
two? The layering probably makes that somewhat ugly :(

We could also just use pg_class.relpages. It'll probably mostly be accurate
enough?

Or we could just cache the result of the last smgrnblocks call...

One of the cases where this type of strategy is most intersting to me is the partial
truncations that autovacuum does... There we even know the range of tables
ahead of time.

Konstantin tested it on various workloads and saw no regression.
But I understand the sentiment on the added overhead on BufferAlloc.
Regarding the case where the patch would potentially affect workloads that fit into
RAM but not into shared buffers, could one of Andres' suggested idea/s above address
that, in addition to this patch's possible shared invalidation fix? Could that settle
the added overhead in BufferAlloc() as temporary fix?

Yes, I think so. Because as far as I can understand he is suggesting
to do changes only in the Truncate/Vacuum code path. Basically, I
think you need to change DropRelFileNodeBuffers or similar functions.
There shouldn't be any change in the BufferAlloc or the common code
path, so there is no question of regression in such cases. I am not
sure what you have in mind for this but feel free to clarify your
understanding before implementation.

Thomas Munro is also working on caching relation sizes [1], maybe that way we
could get the latest known relation size. Currently, it's possible only during
recovery in smgrnblocks.

Tom Lane wrote:

But aside from that, I noted a number of things I didn't like a bit:

* The amount of new shared memory this needs seems several orders of
magnitude higher than what I'd call acceptable: according to my measurements
it's over 10KB per shared buffer! Most of that is going into the
CachedBufTableLock data structure, which seems fundamentally misdesigned ---
how could we be needing a lock per map partition *per buffer*? For comparison,
the space used by buf_table.c is about 56 bytes per shared buffer; I think this
needs to stay at least within hailing distance of there.

* It is fairly suspicious that the new data structure is manipulated while holding
per-partition locks for the existing buffer hashtable.
At best that seems bad for concurrency, and at worst it could result in deadlocks,
because I doubt we can assume that the new hash table has partition boundaries
identical to the old one.

* More generally, it seems like really poor design that this has been written
completely independently of the existing buffer hash table.
Can't we get any benefit by merging them somehow?

The original aim is to just shorten the recovery process, and eventually the speedup
on both vacuum and truncate process are just added bonus.
Given that we don't have a shared invalidation mechanism in place yet like radix tree
buffer mapping which is complex, I thought a patch like mine could be an alternative
approach to that. So I want to improve the patch further.
I hope you can help me clarify the direction, so that I can avoid going farther away
from what the community wants.
1. Both normal operations and recovery process
2. Improve recovery process only

I feel Andres's suggestion will help in both cases.

I wonder if you have considered case of local hash (maintained only during recovery)?
If there is after-crash recovery, then there will be no concurrent
access to shared buffers and this hash will be up-to-date.
in case of hot-standby replica we can use some simple invalidation (just
one flag or counter which indicates that buffer cache was updated).
This hash also can be constructed on demand when DropRelFileNodeBuffers
is called first time (so w have to scan all buffers once, but subsequent
drop operation will be fast.

I'm examining this, but I am not sure if I got the correct understanding. Please correct
me if I'm wrong.
I think above is a suggestion wherein the postgres startup process uses local hash table
to keep track of the buffers of relations. Since there may be other read-only sessions which
read from disk, evict cached blocks, and modify the shared_buffers, the flag would be updated.
We could do it during recovery, then release it as recovery completes.

I haven't looked deeply yet into the source code but we maybe can modify the REDO
(main redo do-while loop) in StartupXLOG() once the read-only connections are consistent.
It would also be beneficial to construct this local hash when DropRefFileNodeBuffers()
is called for the first time, so the whole share buffers is scanned initially, then as
you mentioned subsequent dropping will be fast. (similar behavior to what the patch does)

Do you think this is feasible to be implemented? Or should we explore another approach?

I think we should try what Andres is suggesting as that seems like a
promising idea and can address most of the common problems in this
area but if you feel otherwise, then do let us know.

--
With Regards,
Amit Kapila.

#25Amit Kapila
amit.kapila16@gmail.com
In reply to: Tomas Vondra (#23)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Aug 7, 2020 at 3:03 AM Tomas Vondra
<tomas.vondra@2ndquadrant.com> wrote:

But I understand the sentiment on the added overhead on BufferAlloc.
Regarding the case where the patch would potentially affect workloads
that fit into RAM but not into shared buffers, could one of Andres'
suggested idea/s above address that, in addition to this patch's
possible shared invalidation fix? Could that settle the added overhead
in BufferAlloc() as temporary fix?

Not sure.

Thomas Munro is also working on caching relation sizes [1], maybe that
way we could get the latest known relation size. Currently, it's
possible only during recovery in smgrnblocks.

It's not clear to me how would knowing the relation size help reducing
the overhead of this patch?

AFAICU the idea is to directly call BufTableLookup (similar to how we
do in BufferAlloc) to find the buf_id in function
DropRelFileNodeBuffers and then invalidate the required buffers. And,
we need to do this when the size of the relation is less than some
threshold. So, I think the crux would be to reliably get the number of
blocks information. So, probably relation size cache stuff might help.

--
With Regards,
Amit Kapila.

#26Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#21)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Sat, Aug 1, 2020 at 1:53 AM Andres Freund <andres@anarazel.de> wrote:

Hi,

On 2020-07-31 15:50:04 -0400, Tom Lane wrote:

Andres Freund <andres@anarazel.de> writes:

Wonder if the temporary fix is just to do explicit hashtable probes for
all pages iff the size of the relation is < s_b / 500 or so. That'll
address the case where small tables are frequently dropped - and
dropping large relations is more expensive from the OS and data loading
perspective, so it's not gonna happen as often.

Oooh, interesting idea. We'd need a reliable idea of how long the
relation had been (preferably without adding an lseek call), but maybe
that's do-able.

IIRC we already do smgrnblocks nearby, when doing the truncation (to
figure out which segments we need to remove). Perhaps we can arrange to
combine the two? The layering probably makes that somewhat ugly :(

We could also just use pg_class.relpages. It'll probably mostly be
accurate enough?

Don't we need the accurate 'number of blocks' if we want to invalidate
all the buffers? Basically, I think we need to perform BufTableLookup
for all the blocks in the relation and then Invalidate all buffers.

--
With Regards,
Amit Kapila.

#27Tom Lane
tgl@sss.pgh.pa.us
In reply to: Amit Kapila (#26)
Re: [Patch] Optimize dropping of relation buffers using dlist

Amit Kapila <amit.kapila16@gmail.com> writes:

On Sat, Aug 1, 2020 at 1:53 AM Andres Freund <andres@anarazel.de> wrote:

We could also just use pg_class.relpages. It'll probably mostly be
accurate enough?

Don't we need the accurate 'number of blocks' if we want to invalidate
all the buffers? Basically, I think we need to perform BufTableLookup
for all the blocks in the relation and then Invalidate all buffers.

Yeah, there is no room for "good enough" here. If a dirty buffer remains
in the system, the checkpointer will eventually try to flush it, and fail
(because there's no file to write it to), and then checkpointing will be
stuck. So we cannot afford to risk missing any buffers.

regards, tom lane

#28Konstantin Knizhnik
k.knizhnik@postgrespro.ru
In reply to: Tomas Vondra (#23)
Re: [Patch] Optimize dropping of relation buffers using dlist

On 07.08.2020 00:33, Tomas Vondra wrote:

Unfortunately Konstantin did not share any details about what workloads
he tested, what config etc. But I find the "no regression" hypothesis
rather hard to believe, because we're adding non-trivial amount of code
to a place that can be quite hot.

Sorry, that I have not explained� my test scenarios.
As far as Postgres is pgbench-oriented database:) I have also used pgbench:
read-only case and sip-some updates.
For this patch most critical is number of buffer allocations,
so I used small enough database (scale=100), but shared buffer was set
to 1Gb.
As a result, all data is cached in memory (in file system cache), but
there is intensive swapping at Postgres buffer manager level.
I have tested it both with relatively small (100) and large (1000)
number of clients.
I repeated this tests at my notebook (quadcore, 16Gb RAM, SSD) and IBM
Power2 server with about 380 virtual cores� and about 1Tb of memory.
I the last case results are vary very much I think because of NUMA
architecture) but I failed to find some noticeable regression of patched
version.

But I have to agree that adding parallel hash (in addition to existed
buffer manager hash) is not so good idea.
This cache really quite frequently becomes bottleneck.
My explanation of why I have not observed some noticeable regression was
that this patch uses almost the same lock partitioning schema
as already used it adds not so much new conflicts. May be in case of
POwer2 server, overhead of NUMA is much higher than other factors
(although shared hash is one of the main thing suffering from NUMA
architecture).
But in principle I agree that having two independent caches may decrease
speed up to two times� (or even more).

I hope that everybody will agree that this problem is really critical.
It is certainly not the most common case when there are hundreds of
relation which are frequently truncated. But having quadratic complexity
in drop function is not acceptable from my point of view.
And it is not only recovery-specific problem, this is why solution with
local cache is not enough.

I do not know good solution of the problem. Just some thoughts.
- We can somehow combine locking used for main buffer manager cache (by
relid/blockno) and cache for relid. It will eliminates double locking
overhead.
- We can use something like sorted tree (like std::map) instead of hash
- it will allow to locate blocks both by relid/blockno and by relid only.

#29k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#24)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Friday, August 7, 2020 12:38 PM, Amit Kapila wrote:
Hi,

On Thu, Aug 6, 2020 at 6:53 AM k.jamison@fujitsu.com <k.jamison@fujitsu.com>
wrote:

On Saturday, August 1, 2020 5:24 AM, Andres Freund wrote:

Hi,
Thank you for your constructive review and comments.
Sorry for the late reply.

Hi,

On 2020-07-31 15:50:04 -0400, Tom Lane wrote:

Andres Freund <andres@anarazel.de> writes:

Indeed. The buffer mapping hashtable already is visible as a
major bottleneck in a number of workloads. Even in readonly
pgbench if s_b is large enough (so the hashtable is larger than
the cache). Not to speak of things like a cached sequential scan
with a cheap qual and wide

rows.

To be fair, the added overhead is in buffer allocation not buffer lookup.
So it shouldn't add cost to fully-cached cases. As Tomas noted
upthread, the potential trouble spot is where the working set is
bigger than shared buffers but still fits in RAM (so there's no
actual I/O needed, but we do still have to shuffle buffers a lot).

Oh, right, not sure what I was thinking.

Wonder if the temporary fix is just to do explicit hashtable
probes for all pages iff the size of the relation is < s_b / 500 or so.
That'll address the case where small tables are frequently
dropped - and dropping large relations is more expensive from
the OS and data loading perspective, so it's not gonna happen as often.

Oooh, interesting idea. We'd need a reliable idea of how long the
relation had been (preferably without adding an lseek call), but
maybe that's do-able.

IIRC we already do smgrnblocks nearby, when doing the truncation (to
figure out which segments we need to remove). Perhaps we can arrange
to combine the two? The layering probably makes that somewhat ugly
:(

We could also just use pg_class.relpages. It'll probably mostly be
accurate enough?

Or we could just cache the result of the last smgrnblocks call...

One of the cases where this type of strategy is most intersting to
me is the partial truncations that autovacuum does... There we even
know the range of tables ahead of time.

Konstantin tested it on various workloads and saw no regression.
But I understand the sentiment on the added overhead on BufferAlloc.
Regarding the case where the patch would potentially affect workloads
that fit into RAM but not into shared buffers, could one of Andres'
suggested idea/s above address that, in addition to this patch's
possible shared invalidation fix? Could that settle the added overhead in

BufferAlloc() as temporary fix?

Yes, I think so. Because as far as I can understand he is suggesting to do changes
only in the Truncate/Vacuum code path. Basically, I think you need to change
DropRelFileNodeBuffers or similar functions.
There shouldn't be any change in the BufferAlloc or the common code path, so
there is no question of regression in such cases. I am not sure what you have in
mind for this but feel free to clarify your understanding before implementation.

Thomas Munro is also working on caching relation sizes [1], maybe that
way we could get the latest known relation size. Currently, it's
possible only during recovery in smgrnblocks.

Tom Lane wrote:

But aside from that, I noted a number of things I didn't like a bit:

* The amount of new shared memory this needs seems several orders of
magnitude higher than what I'd call acceptable: according to my
measurements it's over 10KB per shared buffer! Most of that is
going into the CachedBufTableLock data structure, which seems
fundamentally misdesigned --- how could we be needing a lock per map
partition *per buffer*? For comparison, the space used by
buf_table.c is about 56 bytes per shared buffer; I think this needs to stay at

least within hailing distance of there.

* It is fairly suspicious that the new data structure is manipulated
while holding per-partition locks for the existing buffer hashtable.
At best that seems bad for concurrency, and at worst it could result
in deadlocks, because I doubt we can assume that the new hash table
has partition boundaries identical to the old one.

* More generally, it seems like really poor design that this has
been written completely independently of the existing buffer hash table.
Can't we get any benefit by merging them somehow?

The original aim is to just shorten the recovery process, and
eventually the speedup on both vacuum and truncate process are just added

bonus.

Given that we don't have a shared invalidation mechanism in place yet
like radix tree buffer mapping which is complex, I thought a patch
like mine could be an alternative approach to that. So I want to improve the

patch further.

I hope you can help me clarify the direction, so that I can avoid
going farther away from what the community wants.
1. Both normal operations and recovery process 2. Improve recovery
process only

I feel Andres's suggestion will help in both cases.

I wonder if you have considered case of local hash (maintained only during

recovery)?

If there is after-crash recovery, then there will be no concurrent
access to shared buffers and this hash will be up-to-date.
in case of hot-standby replica we can use some simple invalidation
(just one flag or counter which indicates that buffer cache was updated).
This hash also can be constructed on demand when
DropRelFileNodeBuffers is called first time (so w have to scan all
buffers once, but subsequent drop operation will be fast.

I'm examining this, but I am not sure if I got the correct
understanding. Please correct me if I'm wrong.
I think above is a suggestion wherein the postgres startup process
uses local hash table to keep track of the buffers of relations. Since
there may be other read-only sessions which read from disk, evict cached

blocks, and modify the shared_buffers, the flag would be updated.

We could do it during recovery, then release it as recovery completes.

I haven't looked deeply yet into the source code but we maybe can
modify the REDO (main redo do-while loop) in StartupXLOG() once the

read-only connections are consistent.

It would also be beneficial to construct this local hash when
DropRefFileNodeBuffers() is called for the first time, so the whole
share buffers is scanned initially, then as you mentioned subsequent
dropping will be fast. (similar behavior to what the patch does)

Do you think this is feasible to be implemented? Or should we explore another

approach?

I think we should try what Andres is suggesting as that seems like a promising
idea and can address most of the common problems in this area but if you feel
otherwise, then do let us know.

--
With Regards,
Amit Kapila.

Hi, thank you for the review.
I just wanted to confirm so that I can hopefully cover it in the patch revision.
Basically, we don't want the added overhead in BufferAlloc(), so I'll just make
a way to get both the last known relation size and nblocks, and modify the
operations for dropping of relation of buffers, based from the comments
and suggestions of the reviewers. Hopefully I can also provide performance
test results by next CF.

Regards,
Kirk Jamison

#30Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Konstantin Knizhnik (#28)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Aug 07, 2020 at 10:08:23AM +0300, Konstantin Knizhnik wrote:

On 07.08.2020 00:33, Tomas Vondra wrote:

Unfortunately Konstantin did not share any details about what workloads
he tested, what config etc. But I find the "no regression" hypothesis
rather hard to believe, because we're adding non-trivial amount of code
to a place that can be quite hot.

Sorry, that I have not explained� my test scenarios.
As far as Postgres is pgbench-oriented database:) I have also used pgbench:
read-only case and sip-some updates.
For this patch most critical is number of buffer allocations,
so I used small enough database (scale=100), but shared buffer was set
to 1Gb.
As a result, all data is cached in memory (in file system cache), but
there is intensive swapping at Postgres buffer manager level.
I have tested it both with relatively small (100) and large (1000)
number of clients.

I repeated this tests at my notebook (quadcore, 16Gb RAM, SSD) and IBM
Power2 server with about 380 virtual cores� and about 1Tb of memory.
I the last case results are vary very much I think because of NUMA
architecture) but I failed to find some noticeable regression of
patched version.

IMO using such high numbers of clients is pointless - it's perfectly
fine to test just a single client, and the 'basic overhead' should be
visible. It might have some impact on concurrency, but I think that's
just a secondary effect I think. In fact, I wouldn't be surprised if
high client counts made it harder to observe the overhead, due to
concurrency problems (I doubt you have a laptop with this many cores).

Another thing you might try doing is using taskset to attach processes
to particular CPU cores, and also make sure there's no undesirable
influence from CPU power management etc. Laptops are very problematic in
this regard, but even servers can have that enabled in BIOS.

But I have to agree that adding parallel hash (in addition to existed
buffer manager hash) is not so good idea.
This cache really quite frequently becomes bottleneck.
My explanation of why I have not observed some noticeable regression
was that this patch uses almost the same lock partitioning schema
as already used it adds not so much new conflicts. May be in case of
POwer2 server, overhead of NUMA is much higher than other factors
(although shared hash is one of the main thing suffering from NUMA
architecture).
But in principle I agree that having two independent caches may
decrease speed up to two times� (or even more).

I hope that everybody will agree that this problem is really critical.
It is certainly not the most common case when there are hundreds of
relation which are frequently truncated. But having quadratic
complexity in drop function is not acceptable from my point of view.
And it is not only recovery-specific problem, this is why solution
with local cache is not enough.

Well, ultimately it's a balancing act - we need to consider the risk of
regressions vs. how common the improved scenario is. I've seen multiple
applications that e.g. drop many relations (after all, that's why I
optimized that in 9.3) so it's not entirely bogus case.

I do not know good solution of the problem. Just some thoughts.
- We can somehow combine locking used for main buffer manager cache
(by relid/blockno) and cache for relid. It will eliminates double
locking overhead.
- We can use something like sorted tree (like std::map) instead of
hash - it will allow to locate blocks both by relid/blockno and by
relid only.

I don't know. I think the ultimate problem here is that we're adding
code to a fairly hot codepath - it does not matter if it's hash, list,
std::map or something else I think. All of that has overhead.

That's the beauty of Andres' proposal to just loop over the blocks of
the relation and evict them one by one - that adds absolutely nothing to
BufferAlloc.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#31Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#27)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Aug 7, 2020 at 12:03 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Yeah, there is no room for "good enough" here. If a dirty buffer remains
in the system, the checkpointer will eventually try to flush it, and fail
(because there's no file to write it to), and then checkpointing will be
stuck. So we cannot afford to risk missing any buffers.

This comment suggests another possible approach to the problem, which
is to just make a note someplace in shared memory when we drop a
relation. If we later find any of its buffers, we drop them without
writing them out. This is not altogether simple, because (1) we don't
have infinite room in shared memory to accumulate such notes and (2)
it's not impossible for the OID counter to wrap around and permit the
creation of a new relation with the same OID, which would be a problem
if the previous note is still around.

But this might be solvable. Suppose we create a shared hash table
keyed by <dboid, reload> with room for 1 entry per 1000 shared
buffers. When you drop a relation, you insert into the hash table.
Periodically you "clean" the hash table by marking all the entries,
scanning shared buffers to remove any matches, and then deleting all
the marked entries. This should be done periodically in the
background, but if you try to drop a relation and find the hash table
full, or you try to create a relation and find the OID of your new
relation in the hash table, then you have to clean synchronously.

Right now, the cost of dropping K relation when N shared buffers is
O(KN). But with this approach, you only have to incur the O(N)
overhead of scanning shared_buffers when the hash table fills up, and
the hash table size is proportional to N, so the amortized complexity
is O(K); that is, dropping relations takes time proportional to the
number of relations being dropped, but NOT proportional to the size of
shared_buffers, because as shared_buffers grows the hash table gets
proportionally bigger, so that scans don't need to be done as
frequently.

Andres's approach (retail hash table lookups just for blocks less than
the relation size, rather than a full scan) is going to help most with
small relations, whereas this approach helps with relations of any
size, but if you're trying to drop a lot of relations, they're
probably small, and if they are large, scanning shared buffers may not
be the dominant cost, since unlinking the files also takes time. Also,
this approach might turn out to slow down buffer eviction too much.
That could maybe be mitigated by having some kind of cheap fast-path
that gets used when the hash table is empty (like an atomic flag that
indicates whether a hash table probe is needed), and then trying hard
to keep it empty most of the time (e.g. by aggressive background
cleaning, or by ruling that after some number of hash table lookups
the next process to evict a buffer is forced to perform a cleanup).
But you'd probably have to try it to see how well you can do.

It's also possible to combine the two approaches. Small relations
could use Andres's approach while larger ones could use this approach;
or you could insert both large and small relations into this hash
table but use different strategies for cleaning out shared_buffers
depending on the relation size (which could also be preserved in the
hash table).

Just brainstorming here...

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#32Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#31)
Re: [Patch] Optimize dropping of relation buffers using dlist

Robert Haas <robertmhaas@gmail.com> writes:

On Fri, Aug 7, 2020 at 12:03 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Yeah, there is no room for "good enough" here. If a dirty buffer remains
in the system, the checkpointer will eventually try to flush it, and fail
(because there's no file to write it to), and then checkpointing will be
stuck. So we cannot afford to risk missing any buffers.

This comment suggests another possible approach to the problem, which
is to just make a note someplace in shared memory when we drop a
relation. If we later find any of its buffers, we drop them without
writing them out. This is not altogether simple, because (1) we don't
have infinite room in shared memory to accumulate such notes and (2)
it's not impossible for the OID counter to wrap around and permit the
creation of a new relation with the same OID, which would be a problem
if the previous note is still around.

Interesting idea indeed.

As for (1), maybe we don't need to keep the info in shmem. I'll just
point out that the checkpointer has *already got* a complete list of all
recently-dropped relations, cf pendingUnlinks in sync.c. So you could
imagine looking aside at that to discover that a dirty buffer belongs to a
recently-dropped relation. pendingUnlinks would need to be converted to a
hashtable to make searches cheap, and it's not very clear what to do in
backends that haven't got access to that table, but maybe we could just
accept that backends that are forced to flush dirty buffers might do some
useless writes in such cases.

As for (2), the reason why we have that list is that the physical unlink
doesn't happen till after the next checkpoint. So with some messing
around here, we could probably guarantee that every buffer belonging
to the relation has been scanned and deleted before the file unlink
happens --- and then, even if the OID counter has wrapped around, the
OID won't be reassigned to a new relation before that happens.

In short, it seems like maybe we could shove the responsibility for
cleaning up dropped relations' buffers onto the checkpointer without
too much added cost. A possible problem with this is that recycling
of those buffers will happen much more slowly than it does today,
but maybe that's okay?

regards, tom lane

#33Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#32)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Aug 7, 2020 at 12:09 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

As for (1), maybe we don't need to keep the info in shmem. I'll just
point out that the checkpointer has *already got* a complete list of all
recently-dropped relations, cf pendingUnlinks in sync.c. So you could
imagine looking aside at that to discover that a dirty buffer belongs to a
recently-dropped relation. pendingUnlinks would need to be converted to a
hashtable to make searches cheap, and it's not very clear what to do in
backends that haven't got access to that table, but maybe we could just
accept that backends that are forced to flush dirty buffers might do some
useless writes in such cases.

I don't see how that can work. It's not that the writes are useless;
it's that they will fail outright because the file doesn't exist.

As for (2), the reason why we have that list is that the physical unlink
doesn't happen till after the next checkpoint. So with some messing
around here, we could probably guarantee that every buffer belonging
to the relation has been scanned and deleted before the file unlink
happens --- and then, even if the OID counter has wrapped around, the
OID won't be reassigned to a new relation before that happens.

This seems right to me, though.

In short, it seems like maybe we could shove the responsibility for
cleaning up dropped relations' buffers onto the checkpointer without
too much added cost. A possible problem with this is that recycling
of those buffers will happen much more slowly than it does today,
but maybe that's okay?

I suspect it's going to be advantageous to try to make cleaning up
dropped buffers quick in normal cases and allow it to fall behind only
when someone is dropping a lot of relations in quick succession, so
that buffer eviction remains cheap in normal cases. I hadn't thought
about the possible negative performance consequences of failing to put
buffers on the free list, but that's another reason to try to make it
fast.

My viewpoint on this is - I have yet to see anybody really get hosed
because they drop one relation and that causes a full scan of
shared_buffers. I mean, it's slightly expensive, but computers are
fast. Whatever. What hoses people is dropping a lot of relations in
quick succession, either by spamming DROP TABLE commands or by running
something like DROP SCHEMA, and then suddenly they're scanning
shared_buffers over and over again, and their standby is doing the
same thing, and now it hurts. The problem on the standby is actually
worse than the problem on the primary, because the primary can do
other things while one process sits there and thinks about
shared_buffers for a long time, but the standby can't, because the
startup process is all there is.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#34Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#33)
Re: [Patch] Optimize dropping of relation buffers using dlist

Robert Haas <robertmhaas@gmail.com> writes:

On Fri, Aug 7, 2020 at 12:09 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

... it's not very clear what to do in
backends that haven't got access to that table, but maybe we could just
accept that backends that are forced to flush dirty buffers might do some
useless writes in such cases.

I don't see how that can work. It's not that the writes are useless;
it's that they will fail outright because the file doesn't exist.

At least in the case of segment zero, the file will still exist. It'll
have been truncated to zero length, and if the filesystem is stupid about
holes in files then maybe a write to a high block number would consume
excessive disk space, but does anyone still care about such filesystems?
I don't remember at the moment how we handle higher segments, but likely
we could make them still exist too, postponing all the unlinks till after
checkpoint. Or we could just have the backends give up on recycling a
particular buffer if they can't write it (which is the response to an I/O
failure already, I hope).

My viewpoint on this is - I have yet to see anybody really get hosed
because they drop one relation and that causes a full scan of
shared_buffers. I mean, it's slightly expensive, but computers are
fast. Whatever. What hoses people is dropping a lot of relations in
quick succession, either by spamming DROP TABLE commands or by running
something like DROP SCHEMA, and then suddenly they're scanning
shared_buffers over and over again, and their standby is doing the
same thing, and now it hurts.

Yeah, trying to amortize the cost across multiple drops seems like
what we really want here. I'm starting to think about a "relation
dropper" background process, which would be somewhat like the checkpointer
but it wouldn't have any interest in actually doing buffer I/O.
We'd send relation drop commands to it, and it would scan all of shared
buffers and flush related buffers, and then finally do the file truncates
or unlinks. Amortization would happen by considering multiple target
relations during any one scan over shared buffers. I'm not very clear
on how this would relate to the checkpointer's handling of relation
drops, but it could be worked out; if we were lucky maybe the checkpointer
could stop worrying about that.

regards, tom lane

#35Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#34)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Aug 7, 2020 at 12:52 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

At least in the case of segment zero, the file will still exist. It'll
have been truncated to zero length, and if the filesystem is stupid about
holes in files then maybe a write to a high block number would consume
excessive disk space, but does anyone still care about such filesystems?
I don't remember at the moment how we handle higher segments, but likely
we could make them still exist too, postponing all the unlinks till after
checkpoint. Or we could just have the backends give up on recycling a
particular buffer if they can't write it (which is the response to an I/O
failure already, I hope).

None of this sounds very appealing. Postponing the unlinks means
postponing recovery of the space at the OS level, which I think will
be noticeable and undesirable for users. The other notions all seem to
involve treating as valid on-disk states we currently treat as
invalid, and our sanity checks in this area are already far too weak.
And all you're buying for it is putting a hash table that would
otherwise be shared memory into backend-private memory, which seems
like quite a minor gain. Having that information visible to everybody
seems a lot cleaner.

Yeah, trying to amortize the cost across multiple drops seems like
what we really want here. I'm starting to think about a "relation
dropper" background process, which would be somewhat like the checkpointer
but it wouldn't have any interest in actually doing buffer I/O.
We'd send relation drop commands to it, and it would scan all of shared
buffers and flush related buffers, and then finally do the file truncates
or unlinks. Amortization would happen by considering multiple target
relations during any one scan over shared buffers. I'm not very clear
on how this would relate to the checkpointer's handling of relation
drops, but it could be worked out; if we were lucky maybe the checkpointer
could stop worrying about that.

I considered that, too, but it might be overkill. I think that one
scan of shared_buffers every now and then might be cheap enough that
we could just not worry too much about which process gets stuck doing
it. So for example if the number of buffers allocated since the hash
table ended up non-empty reaches NBuffers, the process wanting to do
the next eviction gets handed the job of cleaning it out. Or maybe the
background writer could help; it's not like it does much anyway, zing.
It's possible that a dedicated process is the right solution, but we
might want to at least poke a bit at other alternatives.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#36Amit Kapila
amit.kapila16@gmail.com
In reply to: Tom Lane (#27)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Aug 7, 2020 at 9:33 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Amit Kapila <amit.kapila16@gmail.com> writes:

On Sat, Aug 1, 2020 at 1:53 AM Andres Freund <andres@anarazel.de> wrote:

We could also just use pg_class.relpages. It'll probably mostly be
accurate enough?

Don't we need the accurate 'number of blocks' if we want to invalidate
all the buffers? Basically, I think we need to perform BufTableLookup
for all the blocks in the relation and then Invalidate all buffers.

Yeah, there is no room for "good enough" here. If a dirty buffer remains
in the system, the checkpointer will eventually try to flush it, and fail
(because there's no file to write it to), and then checkpointing will be
stuck. So we cannot afford to risk missing any buffers.

Right, this reminds me of the discussion we had last time on this
topic where we decided that we can't even rely on using smgrnblocks to
find the exact number of blocks because lseek might lie about the EOF
position [1]/messages/by-id/16664.1435414204@sss.pgh.pa.us. So, we anyway need some mechanism to push the
information related to the "to be truncated or dropped relations" to
the background worker (checkpointer and or others) to avoid flush
issues. But, maybe it is better to push the responsibility of
invalidating the buffers for truncated/dropped relation to the
background process. However, I feel for some cases where relation size
is greater than the number of shared buffers there might not be much
benefit in pushing this operation to background unless there are
already a few other relation entries (for dropped relations) so that
cost of scanning the buffers can be amortized.

[1]: /messages/by-id/16664.1435414204@sss.pgh.pa.us

--
With Regards,
Amit Kapila.

#37Amit Kapila
amit.kapila16@gmail.com
In reply to: Robert Haas (#35)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Aug 7, 2020 at 11:03 PM Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Aug 7, 2020 at 12:52 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

At least in the case of segment zero, the file will still exist. It'll
have been truncated to zero length, and if the filesystem is stupid about
holes in files then maybe a write to a high block number would consume
excessive disk space, but does anyone still care about such filesystems?
I don't remember at the moment how we handle higher segments,

We do unlink them and register the request to forget the Fsync
requests for those. See mdunlinkfork.

but likely
we could make them still exist too, postponing all the unlinks till after
checkpoint. Or we could just have the backends give up on recycling a
particular buffer if they can't write it (which is the response to an I/O
failure already, I hope).

Note that we don't often try to flush the buffers from the backend. We
first try to forward the request to checkpoint queue and only if the
queue is full, the backend tries to flush it, so even if we decide to
give up flushing such a buffer (where we get an error) via backend, it
shouldn't impact very many cases. I am not sure but if we can somehow
reliably distinguish this type of error from any other I/O failure
then we can probably give up on flushing this buffer and continue or
maybe just retry to push this request to checkpointer.

None of this sounds very appealing. Postponing the unlinks means
postponing recovery of the space at the OS level, which I think will
be noticeable and undesirable for users. The other notions all seem to
involve treating as valid on-disk states we currently treat as
invalid, and our sanity checks in this area are already far too weak.
And all you're buying for it is putting a hash table that would
otherwise be shared memory into backend-private memory, which seems
like quite a minor gain. Having that information visible to everybody
seems a lot cleaner.

The one more benefit of giving this responsibility to a single process
like checkpointer is that we can avoid unlinking the relation until we
scan all the buffers corresponding to it. Now, surely keeping it in
shared memory and allow other processes to work on it has other merits
which are that such buffers might get invalidated faster but not sure
we can retain the benefit of another approach which is to perform all
such invalidation of buffers before unlinking the relation's first
segment.

--
With Regards,
Amit Kapila.

#38Amit Kapila
amit.kapila16@gmail.com
In reply to: Tom Lane (#27)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Aug 7, 2020 at 9:33 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Amit Kapila <amit.kapila16@gmail.com> writes:

On Sat, Aug 1, 2020 at 1:53 AM Andres Freund <andres@anarazel.de> wrote:

We could also just use pg_class.relpages. It'll probably mostly be
accurate enough?

Don't we need the accurate 'number of blocks' if we want to invalidate
all the buffers? Basically, I think we need to perform BufTableLookup
for all the blocks in the relation and then Invalidate all buffers.

Yeah, there is no room for "good enough" here. If a dirty buffer remains
in the system, the checkpointer will eventually try to flush it, and fail
(because there's no file to write it to), and then checkpointing will be
stuck. So we cannot afford to risk missing any buffers.

Today, again thinking about this point it occurred to me that during
recovery we can reliably find the relation size and after Thomas's
recent commit c5315f4f44 (Cache smgrnblocks() results in recovery), we
might not need to even incur the cost of lseek. Why don't we fix this
first for 'recovery' (by following something on the lines of what
Andres suggested) and then later once we have a generic mechanism for
"caching the relation size" [1]/messages/by-id/CAEepm=3SSw-Ty1DFcK=1rU-K6GSzYzfdD4d+ZwapdN7dTa6=nQ@mail.gmail.com, we can do it for non-recovery paths.
I think that will at least address the reported use case with some
minimal changes.

[1]: /messages/by-id/CAEepm=3SSw-Ty1DFcK=1rU-K6GSzYzfdD4d+ZwapdN7dTa6=nQ@mail.gmail.com

--
With Regards,
Amit Kapila.

#39k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#38)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Tuesday, August 18, 2020 3:05 PM (GMT+9), Amit Kapila wrote:

On Fri, Aug 7, 2020 at 9:33 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Amit Kapila <amit.kapila16@gmail.com> writes:

On Sat, Aug 1, 2020 at 1:53 AM Andres Freund <andres@anarazel.de>

wrote:

We could also just use pg_class.relpages. It'll probably mostly be
accurate enough?

Don't we need the accurate 'number of blocks' if we want to
invalidate all the buffers? Basically, I think we need to perform
BufTableLookup for all the blocks in the relation and then Invalidate all

buffers.

Yeah, there is no room for "good enough" here. If a dirty buffer
remains in the system, the checkpointer will eventually try to flush
it, and fail (because there's no file to write it to), and then
checkpointing will be stuck. So we cannot afford to risk missing any

buffers.

Today, again thinking about this point it occurred to me that during recovery
we can reliably find the relation size and after Thomas's recent commit
c5315f4f44 (Cache smgrnblocks() results in recovery), we might not need to
even incur the cost of lseek. Why don't we fix this first for 'recovery' (by
following something on the lines of what Andres suggested) and then later
once we have a generic mechanism for "caching the relation size" [1], we can
do it for non-recovery paths.
I think that will at least address the reported use case with some minimal
changes.

[1] -
/messages/by-id/CAEepm=3SSw-Ty1DFcK=1r
U-K6GSzYzfdD4d%2BZwapdN7dTa6%3DnQ%40mail.gmail.com

Attached is an updated V9 version with minimal code changes only and
avoids the previous overhead in the BufferAlloc. This time, I only updated
the recovery path as suggested by Amit, and followed Andres' suggestion
of referring to the cached blocks in smgrnblocks.
The layering is kinda tricky so the logic may be wrong. But as of now,
it passes the regression tests. I'll follow up with the performance results.
It seems there's regression for smaller shared_buffers. I'll update if I find bugs.
But I'd also appreciate your reviews in case I missed something.

Regards,
Kirk Jamison

Attachments:

v9-Speedup-dropping-of-relation-buffers-during-recovery.patchapplication/octet-stream; name=v9-Speedup-dropping-of-relation-buffers-during-recovery.patchDownload
From 20b620d4548acc344d9ad065db42e68cc0ce5f14 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k,jamison@fujitsu.com>
Date: Thu, 27 Aug 2020 08:09:39 +0000
Subject: [PATCH] Speedup dropping of relation buffers during recovery

---
 src/backend/storage/buffer/bufmgr.c | 125 +++++++++++++++++++++++++++---------
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   5 +-
 3 files changed, 98 insertions(+), 34 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index a2a963b..95b3c7d 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_THRESHOLD		500	/* NBuffers divided by 2 */
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2979,11 +2981,14 @@ BufferGetLSNAtomic(Buffer buffer)
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2997,44 +3002,102 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	/*
+	 * Proceed to the normal buffer invalidation process. We only speedup
+	 * this path during recovery, because that's the only timing when we
+	 * can get a valid cached value of blocks for relation. See comment
+	 * in smgrnblocks() in smgr.c.
+	 */
+	if (!InRecovery)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		for (i = 0; i < NBuffers; i++)
+		{
+			BufferDesc *bufHdr = GetBufferDescriptor(i);
+			uint32		buf_state;
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag before
+			 * we attempt to lock the buffer; this saves a lot of lock
+			 * acquisitions in typical cases.  It should be safe because the
+			 * caller must have AccessExclusiveLock on the relation, or some other
+			 * reason to be certain that no one is loading new pages of the rel
+			 * into the buffer pool.  (Otherwise we might well miss such pages
+			 * entirely.)  Therefore, while the tag might be changing while we
+			 * look at it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible, and
+			 * false positives are safe because we'll recheck after getting the
+			 * buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but the
+			 * incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
 
-		buf_state = LockBufHdr(bufHdr);
+			buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
+			for (j = 0; j < nforks; j++)
+			{
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= firstDelBlock[j])
+				{
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+					break;
+				}
+			}
+			if (j >= nforks)
+				UnlockBufHdr(bufHdr, buf_state);
+		}
+	}
+	else
+	{
+		BufferTag	newTag;			/* identity of requested block */
+		uint32		newHash;		/* hash value for newTag */
+		LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+		BlockNumber 	reln_nblocks;
+
+		for (i = 0; i < nforks; i++)
 		{
-			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
+			/* Get the number of blocks for the supplied relation fork */
+			reln_nblocks = smgrnblocks(smgr_reln, forkNum[i]);
+
+			/* create a tag so we can lookup the buffer */
+			INIT_BUFFERTAG(newTag, rnode.node, forkNum[i], reln_nblocks);
+
+			/* determine its hash code and partition lock ID */
+			newHash = BufTableHashCode(&newTag);
+			newPartitionLock = BufMappingPartitionLock(newHash);
+
+			if (((int)reln_nblocks) < BUF_DROP_THRESHOLD)
 			{
-				InvalidateBuffer(bufHdr);	/* releases spinlock */
-				break;
+				for (j = 0; j < reln_nblocks; j++)
+				{
+					BufferDesc	*bufHdr;
+					uint32		buf_state;
+					int		buf_id;
+
+					/* Check that it is in the buffer pool */
+					LWLockAcquire(newPartitionLock, LW_SHARED);
+					buf_id = BufTableLookup(&newTag, newHash);
+					LWLockRelease(newPartitionLock);
+
+					bufHdr = GetBufferDescriptor(buf_id);
+
+					if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+						continue;
+
+					buf_state = LockBufHdr(bufHdr);
+
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[j] &&
+						bufHdr->tag.blockNum >= firstDelBlock[j])
+						InvalidateBuffer(bufHdr); /* releases spinlock */
+					else
+						UnlockBufHdr(bufHdr, buf_state);
+				}
 			}
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
 	}
 }
 
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..5238c6c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -583,7 +583,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..da3104c 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -201,9 +201,10 @@ extern BlockNumber RelationGetNumberOfBlocksInFork(Relation relation,
 												   ForkNumber forkNum);
 extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
-extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
+extern void FlushRelationsAllBuffers(struct SMgrRelationData
+ **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

#40Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: k.jamison@fujitsu.com (#39)
Re: [Patch] Optimize dropping of relation buffers using dlist

Hello.

At Tue, 1 Sep 2020 13:02:28 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in

On Tuesday, August 18, 2020 3:05 PM (GMT+9), Amit Kapila wrote:

Today, again thinking about this point it occurred to me that during recovery
we can reliably find the relation size and after Thomas's recent commit
c5315f4f44 (Cache smgrnblocks() results in recovery), we might not need to
even incur the cost of lseek. Why don't we fix this first for 'recovery' (by
following something on the lines of what Andres suggested) and then later
once we have a generic mechanism for "caching the relation size" [1], we can
do it for non-recovery paths.
I think that will at least address the reported use case with some minimal
changes.

[1] -
/messages/by-id/CAEepm=3SSw-Ty1DFcK=1r
U-K6GSzYzfdD4d%2BZwapdN7dTa6%3DnQ%40mail.gmail.com

Isn't a relation always locked asscess-exclusively, at truncation
time? If so, isn't even the result of lseek reliable enough? And if
we don't care the cost of lseek, we can do the same optimization also
for non-recovery paths. Since anyway we perform actual file-truncation
just after so I think the cost of lseek is negligible here.

Attached is an updated V9 version with minimal code changes only and
avoids the previous overhead in the BufferAlloc. This time, I only updated
the recovery path as suggested by Amit, and followed Andres' suggestion
of referring to the cached blocks in smgrnblocks.
The layering is kinda tricky so the logic may be wrong. But as of now,
it passes the regression tests. I'll follow up with the performance results.
It seems there's regression for smaller shared_buffers. I'll update if I find bugs.
But I'd also appreciate your reviews in case I missed something.

BUF_DROP_THRESHOLD seems to be misued. IIUC it defines the maximum
number of file pages that we make relation-targetted search for
buffers. Otherwise we scan through all buffers. On the other hand the
latest patch just leaves all buffers for relation forks longer than
the threshold.

I think we should determine whether to do targetted-scan or full-scan
based on the ratio of (expectedly maximum) total number of pages for
all (specified) forks in a relation against total number of buffers.

By the way

#define BUF_DROP_THRESHOLD 500 /* NBuffers divided by 2 */

NBuffers is not a constant. Even if we wanted to set the macro as
described in the comment, we should have used (NBuffers/2) instead of
"500". But I suppose you might wanted to use (NBuffders / 500) as Tom
suggested upthread. And the name of the macro seems too generic. I
think more specific names like BUF_DROP_FULLSCAN_THRESHOLD would be
better.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#41Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Kyotaro Horiguchi (#40)
Re: [Patch] Optimize dropping of relation buffers using dlist

I'd like make a subtle correction.

At Wed, 02 Sep 2020 10:31:22 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

By the way

#define BUF_DROP_THRESHOLD 500 /* NBuffers divided by 2 */

NBuffers is not a constant. Even if we wanted to set the macro as
described in the comment, we should have used (NBuffers/2) instead of
"500". But I suppose you might wanted to use (NBuffders / 500) as Tom
suggested upthread. And the name of the macro seems too generic. I

Who made the suggestion is Andres, not Tom. Sorry for the mistake.

think more specific names like BUF_DROP_FULLSCAN_THRESHOLD would be
better.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#42Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#40)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Sep 2, 2020 at 7:01 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Hello.

At Tue, 1 Sep 2020 13:02:28 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in

On Tuesday, August 18, 2020 3:05 PM (GMT+9), Amit Kapila wrote:

Today, again thinking about this point it occurred to me that during recovery
we can reliably find the relation size and after Thomas's recent commit
c5315f4f44 (Cache smgrnblocks() results in recovery), we might not need to
even incur the cost of lseek. Why don't we fix this first for 'recovery' (by
following something on the lines of what Andres suggested) and then later
once we have a generic mechanism for "caching the relation size" [1], we can
do it for non-recovery paths.
I think that will at least address the reported use case with some minimal
changes.

[1] -
/messages/by-id/CAEepm=3SSw-Ty1DFcK=1r
U-K6GSzYzfdD4d%2BZwapdN7dTa6%3DnQ%40mail.gmail.com

Isn't a relation always locked asscess-exclusively, at truncation
time? If so, isn't even the result of lseek reliable enough?

Even if the relation is locked, background processes like checkpointer
can still touch the relation which might cause problems. Consider a
case where we extend the relation but didn't flush the newly added
pages. Now during truncate operation, checkpointer can still flush
those pages which can cause trouble for truncate. But, I think in the
recovery path such cases won't cause a problem.

--
With Regards,
Amit Kapila.

#43Tom Lane
tgl@sss.pgh.pa.us
In reply to: Amit Kapila (#42)
Re: [Patch] Optimize dropping of relation buffers using dlist

Amit Kapila <amit.kapila16@gmail.com> writes:

Even if the relation is locked, background processes like checkpointer
can still touch the relation which might cause problems. Consider a
case where we extend the relation but didn't flush the newly added
pages. Now during truncate operation, checkpointer can still flush
those pages which can cause trouble for truncate. But, I think in the
recovery path such cases won't cause a problem.

I wouldn't count on that staying true ...

/messages/by-id/CA+hUKGJ8NRsqgkZEnsnRc2MFROBV-jCnacbYvtpptK2A9YYp9Q@mail.gmail.com

regards, tom lane

#44k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Kyotaro Horiguchi (#40)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wednesday, September 2, 2020 10:31 AM, Kyotaro Horiguchi wrote:

Hello.

At Tue, 1 Sep 2020 13:02:28 +0000, "k.jamison@fujitsu.com"
<k.jamison@fujitsu.com> wrote in

On Tuesday, August 18, 2020 3:05 PM (GMT+9), Amit Kapila wrote:

Today, again thinking about this point it occurred to me that during
recovery we can reliably find the relation size and after Thomas's
recent commit
c5315f4f44 (Cache smgrnblocks() results in recovery), we might not
need to even incur the cost of lseek. Why don't we fix this first
for 'recovery' (by following something on the lines of what Andres
suggested) and then later once we have a generic mechanism for
"caching the relation size" [1], we can do it for non-recovery paths.
I think that will at least address the reported use case with some
minimal changes.

[1] -

/messages/by-id/CAEepm=3SSw-Ty1DFcK=1r

U-K6GSzYzfdD4d%2BZwapdN7dTa6%3DnQ%40mail.gmail.com

Isn't a relation always locked asscess-exclusively, at truncation time? If so,
isn't even the result of lseek reliable enough? And if we don't care the cost of
lseek, we can do the same optimization also for non-recovery paths. Since
anyway we perform actual file-truncation just after so I think the cost of lseek
is negligible here.

The reason for that is when I read the comment in smgrnblocks in smgr.c
I thought that smgrnblocks can only be reliably used during recovery here
to ensure that we have the correct size.
Please correct me if my understanding is wrong, and I'll fix the patch.

* For now, we only use cached values in recovery due to lack of a shared
* invalidation mechanism for changes in file size.
*/
if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
return reln->smgr_cached_nblocks[forknum];

Attached is an updated V9 version with minimal code changes only and
avoids the previous overhead in the BufferAlloc. This time, I only
updated the recovery path as suggested by Amit, and followed Andres'
suggestion of referring to the cached blocks in smgrnblocks.
The layering is kinda tricky so the logic may be wrong. But as of now,
it passes the regression tests. I'll follow up with the performance results.
It seems there's regression for smaller shared_buffers. I'll update if I find

bugs.

But I'd also appreciate your reviews in case I missed something.

BUF_DROP_THRESHOLD seems to be misued. IIUC it defines the maximum
number of file pages that we make relation-targetted search for buffers.
Otherwise we scan through all buffers. On the other hand the latest patch just
leaves all buffers for relation forks longer than the threshold.

Right, I missed the part or condition for that part. Fixed in the latest one.

I think we should determine whether to do targetted-scan or full-scan based
on the ratio of (expectedly maximum) total number of pages for all (specified)
forks in a relation against total number of buffers.

By the way

#define BUF_DROP_THRESHOLD 500 /* NBuffers divided

by 2 */

NBuffers is not a constant. Even if we wanted to set the macro as described
in the comment, we should have used (NBuffers/2) instead of "500". But I
suppose you might wanted to use (NBuffders / 500) as Tom suggested
upthread. And the name of the macro seems too generic. I think more
specific names like BUF_DROP_FULLSCAN_THRESHOLD would be better.

Fixed.

Thank you for the review!
Attached is the v10 of the patch.

Best regards,
Kirk Jamison

Attachments:

v10-Speedup-dropping-of-relation-buffers-during-recovery.patchapplication/octet-stream; name=v10-Speedup-dropping-of-relation-buffers-during-recovery.patchDownload
From 25cac32ee1561c27a0a5eee5ff929d9c44bb1ffc Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 2 Sep 2020 02:57:26 +0000
Subject: [PATCH] Speedup dropping of relation buffers during recovery

---
 src/backend/storage/buffer/bufmgr.c | 148 ++++++++++++++++++++++++++++--------
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 119 insertions(+), 33 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index a2a963b..2b3f08c 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULLSCAN_THRESHOLD		(NBuffers / 2) 	/* NBuffers divided by 2 */
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2979,11 +2981,17 @@ BufferGetLSNAtomic(Buffer buffer)
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	int			k;
+	RelFileNodeBackend	rnode;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2997,44 +3005,122 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	/*
+	 * Proceed to the normal buffer invalidation process. We only speedup
+	 * this path during recovery, because that's the only timing when we
+	 * can get a valid cached value of blocks for relation. See comment
+	 * in smgrnblocks() in smgr.c.
+	 */
+	if (!InRecovery)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		for (i = 0; i < NBuffers; i++)
+		{
+			bufHdr = GetBufferDescriptor(i);
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag before
+			 * we attempt to lock the buffer; this saves a lot of lock
+			 * acquisitions in typical cases.  It should be safe because the
+			 * caller must have AccessExclusiveLock on the relation, or some other
+			 * reason to be certain that no one is loading new pages of the rel
+			 * into the buffer pool.  (Otherwise we might well miss such pages
+			 * entirely.)  Therefore, while the tag might be changing while we
+			 * look at it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible, and
+			 * false positives are safe because we'll recheck after getting the
+			 * buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but the
+			 * incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
 
-		buf_state = LockBufHdr(bufHdr);
+			buf_state = LockBufHdr(bufHdr);
+
+			for (j = 0; j < nforks; j++)
+			{
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= firstDelBlock[j])
+				{
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+					break;
+				}
+			}
+			if (j >= nforks)
+				UnlockBufHdr(bufHdr, buf_state);
+		}
+	}
+	else
+	{
+		BufferTag	newTag;			/* identity of requested block */
+		uint32		newHash;		/* hash value for newTag */
+		LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+		BlockNumber 	reln_nblocks;
 
-		for (j = 0; j < nforks; j++)
+		for (i = 0; i < nforks; i++)
 		{
-			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
+			/* Get the number of blocks for the supplied relation fork */
+			reln_nblocks = smgrnblocks(smgr_reln, forkNum[i]);
+
+			/* create a tag so we can lookup the buffer */
+			INIT_BUFFERTAG(newTag, rnode.node, forkNum[i], reln_nblocks);
+
+			/* determine its hash code and partition lock ID */
+			newHash = BufTableHashCode(&newTag);
+			newPartitionLock = BufMappingPartitionLock(newHash);
+
+			if (((int)reln_nblocks) < BUF_DROP_FULLSCAN_THRESHOLD)
 			{
-				InvalidateBuffer(bufHdr);	/* releases spinlock */
-				break;
+				for (j = 0; j < reln_nblocks; j++)
+				{
+					int		buf_id;
+
+					/* Check that it is in the buffer pool */
+					LWLockAcquire(newPartitionLock, LW_SHARED);
+					buf_id = BufTableLookup(&newTag, newHash);
+					LWLockRelease(newPartitionLock);
+
+					bufHdr = GetBufferDescriptor(buf_id);
+
+					if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+						continue;
+
+					buf_state = LockBufHdr(bufHdr);
+
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[j] &&
+						bufHdr->tag.blockNum >= firstDelBlock[j])
+						InvalidateBuffer(bufHdr); /* releases spinlock */
+					else
+						UnlockBufHdr(bufHdr, buf_state);
+				}
 			}
+			else
+			{
+				for (j = BUF_DROP_FULLSCAN_THRESHOLD; j < NBuffers; j++)
+				{
+					bufHdr = GetBufferDescriptor(j);
+
+					if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+						continue;
+
+					buf_state = LockBufHdr(bufHdr);
+
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[j] &&
+						bufHdr->tag.blockNum >= firstDelBlock[j])
+					{
+						InvalidateBuffer(bufHdr);	/* releases spinlock */
+						break;
+					}
+				}
+			}
+
+			if (i >= nforks)
+				UnlockBufHdr(bufHdr, buf_state);
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
 	}
 }
 
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..5238c6c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -583,7 +583,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

#45Amit Kapila
amit.kapila16@gmail.com
In reply to: Tom Lane (#43)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Sep 2, 2020 at 9:17 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Amit Kapila <amit.kapila16@gmail.com> writes:

Even if the relation is locked, background processes like checkpointer
can still touch the relation which might cause problems. Consider a
case where we extend the relation but didn't flush the newly added
pages. Now during truncate operation, checkpointer can still flush
those pages which can cause trouble for truncate. But, I think in the
recovery path such cases won't cause a problem.

I wouldn't count on that staying true ...

/messages/by-id/CA+hUKGJ8NRsqgkZEnsnRc2MFROBV-jCnacbYvtpptK2A9YYp9Q@mail.gmail.com

I don't think that proposal will matter after commit c5315f4f44
because we are caching the size/blocks for recovery while doing extend
(smgrextend). In the above scenario, we would have cached the blocks
which will be used at later point of time.

--
With Regards,
Amit Kapila.

#46k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#45)
4 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wednesday, September 2, 2020 5:49 PM, Amit Kapila wrote:

On Wed, Sep 2, 2020 at 9:17 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Amit Kapila <amit.kapila16@gmail.com> writes:

Even if the relation is locked, background processes like
checkpointer can still touch the relation which might cause
problems. Consider a case where we extend the relation but didn't
flush the newly added pages. Now during truncate operation,
checkpointer can still flush those pages which can cause trouble for
truncate. But, I think in the recovery path such cases won't cause a

problem.

I wouldn't count on that staying true ...

/messages/by-id/CA+hUKGJ8NRsqgkZEnsnRc2MFR
OBV-jC

nacbYvtpptK2A9YYp9Q@mail.gmail.com

I don't think that proposal will matter after commit c5315f4f44 because we are
caching the size/blocks for recovery while doing extend (smgrextend). In the
above scenario, we would have cached the blocks which will be used at later
point of time.

Hi,

I'm guessing we can still pursue this idea of improving the recovery path first.

I'm working on an updated patch version, because the CFBot's telling
that postgres fails to build (one of the recovery TAP tests fails).
I'm still working on refactoring my patch, but have yet to find a proper solution at the moment.
So I'm going to continue my investigation.

Attached is an updated WIP patch.
I'd appreciate if you could take a look at the patch as well.

In addition, attached also are the regression logs for the failure and other logs
Accompanying it: wal_optimize_node_minimal and wal_optimize_node_replica.

The failures stated in my session was:
t/018_wal_optimize.pl ................ 18/34 Bailout called.
Further testing stopped: pg_ctl start failed
FAILED--Further testing stopped: pg_ctl start failed

Best regards,
Kirk Jamison

Attachments:

v11-Speedup-dropping-of-relation-buffers-during-recovery.patchapplication/octet-stream; name=v11-Speedup-dropping-of-relation-buffers-during-recovery.patchDownload
From 7556544fbbf4e06e94f24f7120437dd7f0b966d4 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 2 Sep 2020 02:57:26 +0000
Subject: [PATCH] Speedup dropping of relation buffers during recovery

---
 src/backend/storage/buffer/bufmgr.c | 144 ++++++++++++++++++++++++++++--------
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 115 insertions(+), 33 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index a2a963b..3dadb83 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULLSCAN_THRESHOLD		(NBuffers / 2)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2979,11 +2981,16 @@ BufferGetLSNAtomic(Buffer buffer)
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2997,44 +3004,119 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	/*
+	 * Proceed to the normal buffer invalidation process. We only speedup
+	 * this path during recovery, because that's the only timing when we
+	 * can get a valid cached value of blocks for relation. See comment
+	 * in smgrnblocks() in smgr.c.
+	 */
+	if (!InRecovery)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		for (i = 0; i < NBuffers; i++)
+		{
+			bufHdr = GetBufferDescriptor(i);
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag before
+			 * we attempt to lock the buffer; this saves a lot of lock
+			 * acquisitions in typical cases.  It should be safe because the
+			 * caller must have AccessExclusiveLock on the relation, or some other
+			 * reason to be certain that no one is loading new pages of the rel
+			 * into the buffer pool.  (Otherwise we might well miss such pages
+			 * entirely.)  Therefore, while the tag might be changing while we
+			 * look at it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible, and
+			 * false positives are safe because we'll recheck after getting the
+			 * buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but the
+			 * incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
 
-		buf_state = LockBufHdr(bufHdr);
+			buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
+			for (j = 0; j < nforks; j++)
+			{
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= firstDelBlock[j])
+				{
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+					break;
+				}
+			}
+			if (j >= nforks)
+				UnlockBufHdr(bufHdr, buf_state);
+		}
+	}
+	else
+	{
+		for (i = 0; i < nforks; i++)
 		{
-			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
+			BlockNumber 	nblocks;
+
+			/* Get the number of blocks for the supplied relation fork */
+			nblocks = smgrnblocks(smgr_reln, forkNum[i]);
+
+			Assert(BlockNumberIsValid(nblocks));
+
+			if (((int)nblocks) < BUF_DROP_FULLSCAN_THRESHOLD)
 			{
-				InvalidateBuffer(bufHdr);	/* releases spinlock */
-				break;
+				for (j = 0; j < (int)nblocks; j++)
+				{
+					int		buf_id;
+					uint32		newHash;		/* hash value for newTag */
+					BufferTag	newTag;			/* identity of requested block */
+					LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+
+					/* create a tag so we can lookup the buffer */
+					INIT_BUFFERTAG(newTag, rnode.node, forkNum[j], firstDelBlock[j]);
+
+					/* determine its hash code and partition lock ID */
+					newHash = BufTableHashCode(&newTag);
+					newPartitionLock = BufMappingPartitionLock(newHash);
+
+					/* Check that it is in the buffer pool */
+					LWLockAcquire(newPartitionLock, LW_SHARED);
+					buf_id = BufTableLookup(&newTag, newHash);
+					LWLockRelease(newPartitionLock);
+
+					bufHdr = GetBufferDescriptor(buf_id);
+
+					if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+						continue;
+
+					buf_state = LockBufHdr(bufHdr);
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[j] &&
+						bufHdr->tag.blockNum >= firstDelBlock[j])
+						InvalidateBuffer(bufHdr); /* releases spinlock */
+					else
+						UnlockBufHdr(bufHdr, buf_state);
+				}
+			}
+			else
+			{
+				for (j = BUF_DROP_FULLSCAN_THRESHOLD; j < NBuffers; j++)
+				{
+					bufHdr = GetBufferDescriptor(j);
+
+					if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+						continue;
+
+					buf_state = LockBufHdr(bufHdr);
+
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[j] &&
+						bufHdr->tag.blockNum >= firstDelBlock[j])
+						InvalidateBuffer(bufHdr); /* releases spinlock */
+					else
+						UnlockBufHdr(bufHdr, buf_state);
+				}
 			}
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
 	}
 }
 
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..5238c6c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -583,7 +583,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

regress_log_018_wal_optimizeapplication/octet-stream; name=regress_log_018_wal_optimizeDownload
018_wal_optimize_node_minimal.logapplication/octet-stream; name=018_wal_optimize_node_minimal.logDownload
018_wal_optimize_node_replica.logapplication/octet-stream; name=018_wal_optimize_node_replica.logDownload
#47Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#46)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Mon, Sep 7, 2020 at 1:33 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Wednesday, September 2, 2020 5:49 PM, Amit Kapila wrote:

On Wed, Sep 2, 2020 at 9:17 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Amit Kapila <amit.kapila16@gmail.com> writes:

Even if the relation is locked, background processes like
checkpointer can still touch the relation which might cause
problems. Consider a case where we extend the relation but didn't
flush the newly added pages. Now during truncate operation,
checkpointer can still flush those pages which can cause trouble for
truncate. But, I think in the recovery path such cases won't cause a

problem.

I wouldn't count on that staying true ...

/messages/by-id/CA+hUKGJ8NRsqgkZEnsnRc2MFR
OBV-jC

nacbYvtpptK2A9YYp9Q@mail.gmail.com

I don't think that proposal will matter after commit c5315f4f44 because we are
caching the size/blocks for recovery while doing extend (smgrextend). In the
above scenario, we would have cached the blocks which will be used at later
point of time.

I'm guessing we can still pursue this idea of improving the recovery path first.

I think so.

I'm working on an updated patch version, because the CFBot's telling
that postgres fails to build (one of the recovery TAP tests fails).
I'm still working on refactoring my patch, but have yet to find a proper solution at the moment.
So I'm going to continue my investigation.

Attached is an updated WIP patch.
I'd appreciate if you could take a look at the patch as well.

So, I see the below log as one of the problems:
2020-09-07 06:20:33.918 UTC [10914] LOG: redo starts at 0/15FFEC0
2020-09-07 06:20:33.919 UTC [10914] FATAL: unexpected data beyond EOF
in block 1 of relation base/13743/24581

This indicates that we missed invalidating some buffer which should
have been invalidated. If you are able to reproduce this locally then
I suggest to first write a simple patch without the check of the
threshold, basically in recovery always try to use the new way to
invalidate the buffer. That will reduce the scope of the code that can
create a problem. Let us know if the problem still exists and share
the logs. BTW, I think I see one problem in the code:

if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+ bufHdr->tag.forkNum == forkNum[j] &&
+ bufHdr->tag.blockNum >= firstDelBlock[j])

Here, I think you need to use 'i' not 'j' for forkNum and
firstDelBlock as those are arrays w.r.t forks. That might fix the
problem but I am not sure as I haven't tried to reproduce it.

--
With Regards,
Amit Kapila.

#48tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Amit Kapila (#47)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Amit Kapila <amit.kapila16@gmail.com>

if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+ bufHdr->tag.forkNum == forkNum[j] &&
+ bufHdr->tag.blockNum >= firstDelBlock[j])

Here, I think you need to use 'i' not 'j' for forkNum and
firstDelBlock as those are arrays w.r.t forks. That might fix the
problem but I am not sure as I haven't tried to reproduce it.

(1)
+ INIT_BUFFERTAG(newTag, rnode.node, forkNum[j], firstDelBlock[j]);

And you need to use i here, too.

I advise you to suspect any character, any word, and any sentence. I've found many bugs for others so far. I'm afraid you're just seeing the code flow.

(2)
+					LWLockAcquire(newPartitionLock, LW_SHARED);
+					buf_id = BufTableLookup(&newTag, newHash);
+					LWLockRelease(newPartitionLock);
+
+					bufHdr = GetBufferDescriptor(buf_id);

Check the result of BufTableLookup() and do nothing if the block is not in the shared buffers.

(3)
+			else
+			{
+				for (j = BUF_DROP_FULLSCAN_THRESHOLD; j < NBuffers; j++)
+				{

What's the meaning of this loop? I don't understand the start condition. Should j be initialized to 0?

(4)
+#define BUF_DROP_FULLSCAN_THRESHOLD (NBuffers / 2)

Wasn't it 500 instead of 2? Anyway, I think we need to discuss this threshold later.

(5)
+ if (((int)nblocks) < BUF_DROP_FULLSCAN_THRESHOLD)

It's better to define BUF_DROP_FULLSCAN_THRESHOLD as an uint32 value instead of casting the type here, as these values are blocks.

Regards
Takayuki Tsunakawa

#49tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#48)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: tsunakawa.takay@fujitsu.com <tsunakawa.takay@fujitsu.com>

(1)
+ INIT_BUFFERTAG(newTag,
rnode.node, forkNum[j], firstDelBlock[j]);

And you need to use i here, too.

I remember the books "Code Complete" and/or "Readable Code" suggest to use meaningful loop variable names like fork_num and block_count, to prevent this type of mistakes.

Regards
Takayuki Tsunakawa

#50k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#47)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Tuesday, September 8, 2020 1:02 PM, Amit Kapila wrote:
Hello,

On Mon, Sep 7, 2020 at 1:33 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Wednesday, September 2, 2020 5:49 PM, Amit Kapila wrote:

On Wed, Sep 2, 2020 at 9:17 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Amit Kapila <amit.kapila16@gmail.com> writes:

Even if the relation is locked, background processes like
checkpointer can still touch the relation which might cause
problems. Consider a case where we extend the relation but
didn't flush the newly added pages. Now during truncate
operation, checkpointer can still flush those pages which can
cause trouble for truncate. But, I think in the recovery path
such cases won't cause a

problem.

I wouldn't count on that staying true ...

/messages/by-id/CA+hUKGJ8NRsqgkZEnsnRc2MFR

OBV-jC

nacbYvtpptK2A9YYp9Q@mail.gmail.com

I don't think that proposal will matter after commit c5315f4f44
because we are caching the size/blocks for recovery while doing
extend (smgrextend). In the above scenario, we would have cached the
blocks which will be used at later point of time.

I'm guessing we can still pursue this idea of improving the recovery path

first.

I think so.

Alright, so I've updated the patch which passes the regression and TAP tests.
It compiles and builds as intended.

I'm working on an updated patch version, because the CFBot's telling
that postgres fails to build (one of the recovery TAP tests fails).
I'm still working on refactoring my patch, but have yet to find a proper

solution at the moment.

So I'm going to continue my investigation.

Attached is an updated WIP patch.
I'd appreciate if you could take a look at the patch as well.

So, I see the below log as one of the problems:
2020-09-07 06:20:33.918 UTC [10914] LOG: redo starts at 0/15FFEC0
2020-09-07 06:20:33.919 UTC [10914] FATAL: unexpected data beyond EOF
in block 1 of relation base/13743/24581

This indicates that we missed invalidating some buffer which should have
been invalidated. If you are able to reproduce this locally then I suggest to first
write a simple patch without the check of the threshold, basically in recovery
always try to use the new way to invalidate the buffer. That will reduce the
scope of the code that can create a problem. Let us know if the problem still
exists and share the logs. BTW, I think I see one problem in the code:

if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+ bufHdr->tag.forkNum == forkNum[j] && tag.blockNum >= 
+ bufHdr->firstDelBlock[j])

Here, I think you need to use 'i' not 'j' for forkNum and
firstDelBlock as those are arrays w.r.t forks. That might fix the
problem but I am not sure as I haven't tried to reproduce it.

Thanks for advice. Right, that seems to be the cause of error,
and fixing that (using fork) solved the case.
I also followed the advice of Tsunakawa-san of using more meaningful iterator
Instead of using "i" & "j" for readability.

I also added a new function when relation fork is bigger than the threshold
If (nblocks > BUF_DROP_FULLSCAN_THRESHOLD)
(DropRelFileNodeBuffersOfFork) Perhaps there's a better name for that function.
However, as expected in the previous discussions, this is a bit slower than the
standard buffer invalidation process, because the whole shared buffers are scanned nfork times.
Currently, I set the threshold to (NBuffers / 500)

Feedback on the patch/testing are very much welcome.

Best regards,
Kirk Jamison

Attachments:

v12-Speedup-dropping-of-relation-buffers-during-recovery.patchapplication/octet-stream; name=v12-Speedup-dropping-of-relation-buffers-during-recovery.patchDownload
From 83d01f27df5f6128f5abadac501d76a282c4cd36 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 11 Sep 2020 13:00:33 +0000
Subject: [PATCH] Speedup dropping of relation buffers during recovery

---
 src/backend/storage/buffer/bufmgr.c | 173 ++++++++++++++++++++++++++++++------
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 146 insertions(+), 31 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index a2a963b..01ea4d5 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULLSCAN_THRESHOLD		(uint32)(NBuffers / 500)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,8 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+										 BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2979,65 +2983,176 @@ BufferGetLSNAtomic(Buffer buffer)
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
-	int			i;
-	int			j;
+	int			buf_num;
+	int			fork_num;
+	RelFileNodeBackend	rnode;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
 	{
 		if (rnode.backend == MyBackendId)
 		{
-			for (j = 0; j < nforks; j++)
-				DropRelFileNodeLocalBuffers(rnode.node, forkNum[j],
-											firstDelBlock[j]);
+			for (fork_num = 0; fork_num < nforks; fork_num++)
+				DropRelFileNodeLocalBuffers(rnode.node, forkNum[fork_num],
+											firstDelBlock[fork_num]);
 		}
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	/*
+	 * We only speedup this path during recovery, because that's the only
+	 * timing when we can get a valid cached value of blocks for relation.
+	 * See comment in smgrnblocks() in smgr.c. Otherwise, proceed to usual
+	 * buffer invalidation process (scanning of whole shared buffers).
+	 */
+	if (InRecovery)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		for (fork_num = 0; fork_num < nforks; fork_num++)
+		{
+			BlockNumber 	nblocks;
+
+			/* Get the number of blocks for the supplied relation's fork */
+			nblocks = smgrnblocks(smgr_reln, forkNum[fork_num]);
+			Assert(BlockNumberIsValid(nblocks));
+
+			if (nblocks < BUF_DROP_FULLSCAN_THRESHOLD)
+			{
+				BlockNumber		block_num;
+				for (block_num = 0; block_num < nblocks; block_num++)
+				{
+					uint32		newHash;		/* hash value for newTag */
+					BufferTag	newTag;			/* identity of requested block */
+					LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+					int		buf_id;
+
+					/* create a tag so we can lookup the buffer */
+					INIT_BUFFERTAG(newTag, rnode.node, forkNum[fork_num],
+								   firstDelBlock[fork_num]);
+
+					/* determine its hash code and partition lock ID */
+					newHash = BufTableHashCode(&newTag);
+					newPartitionLock = BufMappingPartitionLock(newHash);
+
+					/* Check that it is in the buffer pool. If not, do nothing */
+					LWLockAcquire(newPartitionLock, LW_SHARED);
+					buf_id = BufTableLookup(&newTag, newHash);
+
+					if (buf_id < 0)
+					{
+						LWLockRelease(newPartitionLock);
+						continue;
+					}
+					LWLockRelease(newPartitionLock);
+
+					bufHdr = GetBufferDescriptor(buf_id);
+
+					/*
+					 * We can make this a tad faster by prechecking the buffer tag before
+					 * we attempt to lock the buffer; this saves a lot of lock
+					 * acquisitions in typical cases.  It should be safe because the
+					 * caller must have AccessExclusiveLock on the relation, or some other
+					 * reason to be certain that no one is loading new pages of the rel
+					 * into the buffer pool.  (Otherwise we might well miss such pages
+					 * entirely.)  Therefore, while the tag might be changing while we
+					 * look at it, it can't be changing *to* a value we care about, only
+					 * *away* from such a value.  So false negatives are impossible, and
+					 * false positives are safe because we'll recheck after getting the
+					 * buffer lock.
+					 *
+					 * We could check forkNum and blockNum as well as the rnode, but the
+					 * incremental win from doing so seems small.
+					 */
+					if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+						continue;
+
+					buf_state = LockBufHdr(bufHdr);
+
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[fork_num] &&
+						bufHdr->tag.blockNum >= firstDelBlock[fork_num])
+						InvalidateBuffer(bufHdr);	/* releases spinlock */
+					else
+						UnlockBufHdr(bufHdr, buf_state);
+				}
+				if (block_num >= nblocks)
+				{
+					DropRelFileNodeBuffersOfFork(rnode.node, forkNum[fork_num],
+												firstDelBlock[fork_num]);
+				}
+			}
+			else
+			{
+				DropRelFileNodeBuffersOfFork(rnode.node, forkNum[fork_num],
+											 firstDelBlock[fork_num]);
+			}
+		}
+		return;
+	}
+	for (buf_num = 0; buf_num < NBuffers; buf_num++)
+	{
+		bufHdr = GetBufferDescriptor(buf_num);
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
 		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
 			continue;
 
 		buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
+		for (fork_num = 0; fork_num < nforks; fork_num++)
 		{
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
+				bufHdr->tag.forkNum == forkNum[fork_num] &&
+				bufHdr->tag.blockNum >= firstDelBlock[fork_num])
 			{
 				InvalidateBuffer(bufHdr);	/* releases spinlock */
 				break;
 			}
 		}
-		if (j >= nforks)
+		if (fork_num >= nforks)
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+
+/* -----------------------------------------------------------------
+ *		DropRelFileNodeBuffersOfFork
+ *
+ *		This function removes from the buffer pool the pages for
+ *		each fork of the specified relation.
+ * -----------------------------------------------------------------
+ */
+static void
+DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+							 BlockNumber firstDelBlock)
+{
+	int			buf_num;
+
+	for (buf_num = 0; buf_num < NBuffers; buf_num++)
+	{
+		BufferDesc	*bufHdr = GetBufferDescriptor(buf_num);
+		uint32		buf_state;
+
+		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode))
+			continue;
+
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
 			UnlockBufHdr(bufHdr, buf_state);
 	}
 }
 
+
 /* ---------------------------------------------------------------------
  *		DropRelFileNodesAllBuffers
  *
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..5238c6c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -583,7 +583,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

#51k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#50)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi,

BTW, I think I see one problem in the code:

if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+ bufHdr->tag.forkNum == forkNum[j] && tag.blockNum >=
+ bufHdr->firstDelBlock[j])

Here, I think you need to use 'i' not 'j' for forkNum and
firstDelBlock as those are arrays w.r.t forks. That might fix the
problem but I am not sure as I haven't tried to reproduce it.

Thanks for advice. Right, that seems to be the cause of error, and fixing that
(using fork) solved the case.
I also followed the advice of Tsunakawa-san of using more meaningful
iterator Instead of using "i" & "j" for readability.

I also added a new function when relation fork is bigger than the threshold
If (nblocks > BUF_DROP_FULLSCAN_THRESHOLD)
(DropRelFileNodeBuffersOfFork) Perhaps there's a better name for that
function.
However, as expected in the previous discussions, this is a bit slower than the
standard buffer invalidation process, because the whole shared buffers are
scanned nfork times.
Currently, I set the threshold to (NBuffers / 500)

I made a mistake in the v12. I replaced the firstDelBlock[fork_num] with firstDelBlock[block_num],
In the for-loop code block of block_num, because we want to process the current block of per-block loop

OTOH, I used the firstDelBlock[fork_num] when relation fork is bigger than the threshold,
or if the cached blocks of small relations were already invalidated.

The logic could be either correct or wrong, so I'd appreciate feedback and comments/advice.

Regards,
Kirk Jamison

Attachments:

v13-Speedup-dropping-of-relation-buffers-during-recovery.patchapplication/octet-stream; name=v13-Speedup-dropping-of-relation-buffers-during-recovery.patchDownload
From f670d8135cb220e96eac8037912a92c4a8eb3eb1 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 11 Sep 2020 13:00:33 +0000
Subject: [PATCH] Speedup dropping of relation buffers during recovery

Added ereport/elog for debugging
---
 src/backend/storage/buffer/bufmgr.c | 182 ++++++++++++++++++++++++++++++------
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 155 insertions(+), 31 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index a2a963b..bd1d89b 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULLSCAN_THRESHOLD		(uint32)(NBuffers / 500)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,8 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+										 BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2979,65 +2983,185 @@ BufferGetLSNAtomic(Buffer buffer)
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
-	int			i;
-	int			j;
+	int			buf_num;
+	int			fork_num;
+	RelFileNodeBackend	rnode;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
 	{
 		if (rnode.backend == MyBackendId)
 		{
-			for (j = 0; j < nforks; j++)
-				DropRelFileNodeLocalBuffers(rnode.node, forkNum[j],
-											firstDelBlock[j]);
+			for (fork_num = 0; fork_num < nforks; fork_num++)
+				DropRelFileNodeLocalBuffers(rnode.node, forkNum[fork_num],
+											firstDelBlock[fork_num]);
 		}
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	/*
+	 * We only speedup this path during recovery, because that's the only
+	 * timing when we can get a valid cached value of blocks for relation.
+	 * See comment in smgrnblocks() in smgr.c. Otherwise, proceed to usual
+	 * buffer invalidation process (scanning of whole shared buffers).
+	 */
+	if (InRecovery)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		for (fork_num = 0; fork_num < nforks; fork_num++)
+		{
+			BlockNumber 	nblocks;
+
+			/* Get the number of blocks for the supplied relation's fork */
+			nblocks = smgrnblocks(smgr_reln, forkNum[fork_num]);
+			Assert(BlockNumberIsValid(nblocks));
+
+			if (nblocks < BUF_DROP_FULLSCAN_THRESHOLD)
+			{
+				BlockNumber		block_num;
+
+				for (block_num = 0; block_num <= nblocks; block_num++)
+				{
+					uint32		newHash;		/* hash value for newTag */
+					BufferTag	newTag;			/* identity of requested block */
+					LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+					int		buf_id;
+
+					/* create a tag with respect to the block so we can lookup the buffer */
+					INIT_BUFFERTAG(newTag, rnode.node, forkNum[fork_num],
+								   firstDelBlock[block_num]);
+
+					/* determine its hash code and partition lock ID */
+					newHash = BufTableHashCode(&newTag);
+					newPartitionLock = BufMappingPartitionLock(newHash);
+
+					/* Check that it is in the buffer pool. If not, do nothing */
+					LWLockAcquire(newPartitionLock, LW_SHARED);
+					buf_id = BufTableLookup(&newTag, newHash);
+
+					if (buf_id < 0)
+					{
+						LWLockRelease(newPartitionLock);
+						continue;
+					}
+					LWLockRelease(newPartitionLock);
+
+					bufHdr = GetBufferDescriptor(buf_id);
+
+					/*
+					 * We can make this a tad faster by prechecking the buffer tag before
+					 * we attempt to lock the buffer; this saves a lot of lock
+					 * acquisitions in typical cases.  It should be safe because the
+					 * caller must have AccessExclusiveLock on the relation, or some other
+					 * reason to be certain that no one is loading new pages of the rel
+					 * into the buffer pool.  (Otherwise we might well miss such pages
+					 * entirely.)  Therefore, while the tag might be changing while we
+					 * look at it, it can't be changing *to* a value we care about, only
+					 * *away* from such a value.  So false negatives are impossible, and
+					 * false positives are safe because we'll recheck after getting the
+					 * buffer lock.
+					 *
+					 * We could check forkNum and blockNum as well as the rnode, but the
+					 * incremental win from doing so seems small.
+					 */
+					if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+						continue;
+
+					buf_state = LockBufHdr(bufHdr);
+
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[fork_num] &&
+						bufHdr->tag.blockNum >= firstDelBlock[block_num])
+						InvalidateBuffer(bufHdr);	/* releases spinlock */
+					else
+						UnlockBufHdr(bufHdr, buf_state);
+				}
+				/*
+				 * We've invalidated the nblocks already. Scan the shared buffers
+				 * for each fork.
+				 */
+				if (block_num > nblocks)
+				{
+					DropRelFileNodeBuffersOfFork(rnode.node, forkNum[fork_num],
+												 firstDelBlock[fork_num]);
+				}
+			}
+			else
+			{
+				/*
+				 * Relation fork is bigger than the threshold. Scan the shared buffers
+				 * for each fork.
+				 */
+				DropRelFileNodeBuffersOfFork(rnode.node, forkNum[fork_num],
+											 firstDelBlock[fork_num]);
+			}
+		}
+		return;
+	}
+	for (buf_num = 0; buf_num < NBuffers; buf_num++)
+	{
+		bufHdr = GetBufferDescriptor(buf_num);
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
 		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
 			continue;
 
 		buf_state = LockBufHdr(bufHdr);
 
-		for (j = 0; j < nforks; j++)
+		for (fork_num = 0; fork_num < nforks; fork_num++)
 		{
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
+				bufHdr->tag.forkNum == forkNum[fork_num] &&
+				bufHdr->tag.blockNum >= firstDelBlock[fork_num])
 			{
 				InvalidateBuffer(bufHdr);	/* releases spinlock */
 				break;
 			}
 		}
-		if (j >= nforks)
+		if (fork_num >= nforks)
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+
+/* -----------------------------------------------------------------
+ *		DropRelFileNodeBuffersOfFork
+ *
+ *		This function removes from the buffer pool the pages for
+ *		each fork of the specified relation.
+ * -----------------------------------------------------------------
+ */
+static void
+DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+							 BlockNumber firstDelBlock)
+{
+	int			buf_num;
+
+	for (buf_num = 0; buf_num < NBuffers; buf_num++)
+	{
+		BufferDesc	*bufHdr = GetBufferDescriptor(buf_num);
+		uint32		buf_state;
+
+		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode))
+			continue;
+
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
 			UnlockBufHdr(bufHdr, buf_state);
 	}
 }
 
+
 /* ---------------------------------------------------------------------
  *		DropRelFileNodesAllBuffers
  *
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..5238c6c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -583,7 +583,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

#52Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#42)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Wed, 2 Sep 2020 08:18:06 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Wed, Sep 2, 2020 at 7:01 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Isn't a relation always locked asscess-exclusively, at truncation
time? If so, isn't even the result of lseek reliable enough?

Even if the relation is locked, background processes like checkpointer
can still touch the relation which might cause problems. Consider a
case where we extend the relation but didn't flush the newly added
pages. Now during truncate operation, checkpointer can still flush
those pages which can cause trouble for truncate. But, I think in the
recovery path such cases won't cause a problem.

I reconsided on this and still have a doubt.

Is this means lseek(SEEK_END) doesn't count blocks that are
write(2)'ed (by smgrextend) but not yet flushed? (I don't think so,
for clarity.) The nblocks cache is added just to reduce the number of
lseek()s and expected to always have the same value with what lseek()
is expected to return. The reason it is reliable only during recovery
is that the cache is not shared but the startup process is the only
process that changes the relation size during recovery.

If any other process can extend the relation while smgrtruncate is
running, the current DropRelFileNodeBuffers should have the chance
that a new buffer for extended area is allocated at a buffer location
where the function already have passed by, which is a disaster.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#53tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#51)
RE: [Patch] Optimize dropping of relation buffers using dlist

The code doesn't seem to be working correctly.

(1)
+ for (block_num = 0; block_num <= nblocks; block_num++)

should be

+ for (block_num = firstDelBlock[fork_num]; block_num < nblocks; block_num++)

because:

* You only want to invalidate blocks >= firstDelBlock[fork_num], don't you?
* The relation's block number ranges from 0 to nblocks - 1.

(2)
+					INIT_BUFFERTAG(newTag, rnode.node, forkNum[fork_num],
+								   firstDelBlock[block_num]);

Replace firstDelBlock[fork_num] with block_num, because you want to process the current block of per-block loop. Your code accesses memory out of the bounds of the array, and doesn't invalidate any buffer.

(3)
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[fork_num] &&
+						bufHdr->tag.blockNum >= firstDelBlock[block_num])
+						InvalidateBuffer(bufHdr);	/* releases spinlock */
+					else
+						UnlockBufHdr(bufHdr, buf_state);

Replace
bufHdr->tag.blockNum >= firstDelBlock[fork_num]
with
bufHdr->tag.blockNum == block_num
because you want to check if the found buffer is for the current block of the loop.

(4)
+				/*
+				 * We've invalidated the nblocks already. Scan the shared buffers
+				 * for each fork.
+				 */
+				if (block_num > nblocks)
+				{
+					DropRelFileNodeBuffersOfFork(rnode.node, forkNum[fork_num],
+												 firstDelBlock[fork_num]);
+				}

This part is unnecessary. This invalidates all buffers that (2) failed to process, so the regression test succeeds.

Regards
Takayuki Tsunakawa

#54Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: k.jamison@fujitsu.com (#51)
Re: [Patch] Optimize dropping of relation buffers using dlist

Thanks for the new version. Jamison.

At Tue, 15 Sep 2020 11:11:26 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in

Hi,

BTW, I think I see one problem in the code:

if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+ bufHdr->tag.forkNum == forkNum[j] && tag.blockNum >=
+ bufHdr->firstDelBlock[j])

Here, I think you need to use 'i' not 'j' for forkNum and
firstDelBlock as those are arrays w.r.t forks. That might fix the
problem but I am not sure as I haven't tried to reproduce it.

Thanks for advice. Right, that seems to be the cause of error, and fixing that
(using fork) solved the case.
I also followed the advice of Tsunakawa-san of using more meaningful
iterator Instead of using "i" & "j" for readability.

(FWIW, I prefer short conventional names for short-term iterator variables.)

master> * XXX currently it sequentially searches the buffer pool, should be
master> * changed to more clever ways of searching. However, this routine
master> * is used only in code paths that aren't very performance-critical,
master> * and we shouldn't slow down the hot paths to make it faster ...

This comment needs a rewrite.

+		for (fork_num = 0; fork_num < nforks; fork_num++)
 		{
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
+				bufHdr->tag.forkNum == forkNum[fork_num] &&
+				bufHdr->tag.blockNum >= firstDelBlock[fork_num])

fork_num is not actually a fork number, but the index of forkNum[].
It should be fork_idx (or just i, which I prefer..).

-			for (j = 0; j < nforks; j++)
-				DropRelFileNodeLocalBuffers(rnode.node, forkNum[j],
-											firstDelBlock[j]);
+			for (fork_num = 0; fork_num < nforks; fork_num++)
+				DropRelFileNodeLocalBuffers(rnode.node, forkNum[fork_num],
+											firstDelBlock[fork_num]);

I think we don't need to include the irrelevant refactoring in this
patch. (And I think j is better there.)

+	 * We only speedup this path during recovery, because that's the only
+	 * timing when we can get a valid cached value of blocks for relation.
+	 * See comment in smgrnblocks() in smgr.c. Otherwise, proceed to usual
+	 * buffer invalidation process (scanning of whole shared buffers).

We need an explanation of why we do this optimizaton only for the
recovery case.

+			/* Get the number of blocks for the supplied relation's fork */
+			nblocks = smgrnblocks(smgr_reln, forkNum[fork_num]);
+			Assert(BlockNumberIsValid(nblocks));
+
+			if (nblocks < BUF_DROP_FULLSCAN_THRESHOLD)

As mentioned upthread, the criteria whether we do full-scan or
lookup-drop is how large portion of NBUFFERS this relation-drop can be
going to invalidate. So the nblocks above sould be the sum of number
of blocks to be truncated (not just the total number of blocks) of all
designated forks. Then once we decided to do loopup-drop method, we
do that for all forks.

+				for (block_num = 0; block_num <= nblocks; block_num++)
+				{

block_num is quite confusing with nblocks, at least for
me(:p). Likewise fork_num, I prefer that it is just j or iblk or
something else anyway not confusing with nblocks. By the way, the
loop runs nblocks + 1 times, which seems wrong. We can start the loop
from firstDelBlock[fork_num], instead of 0 and that makes the check
against firstDelBlock[] later useless.

+					/* create a tag with respect to the block so we can lookup the buffer */
+					INIT_BUFFERTAG(newTag, rnode.node, forkNum[fork_num],
+								   firstDelBlock[block_num]);

Mmm. it is wrong that the tag is initialized using
firstDelBlock[block_num]. Why isn't is just block_num?

+					if (buf_id < 0)
+					{
+						LWLockRelease(newPartitionLock);
+						continue;
+					}
+					LWLockRelease(newPartitionLock);

We don't need two separate LWLockRelease()'s there.

+   /*
+    * We can make this a tad faster by prechecking the buffer tag before
+    * we attempt to lock the buffer; this saves a lot of lock
...
+    */
+   if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+   	continue;

In the original code, this is performed in order to avoid taking a
lock on bufHder for irrelevant buffers. We have identified the buffer
by looking up using the rnode, so I think we don't need to this
check. Note that we are doing the same check after lock aquisition.

+    	else
+    		UnlockBufHdr(bufHdr, buf_state);
+    }
+    /*
+     * We've invalidated the nblocks already. Scan the shared buffers
+     * for each fork.
+     */
+    if (block_num > nblocks)
+    {
+    	DropRelFileNodeBuffersOfFork(rnode.node, forkNum[fork_num],
+    								 firstDelBlock[fork_num]);
+    }

Mmm? block_num is always larger than nblocks there. And the function
call runs a whole Nbuffers scan for the just-processed fork. What is
the point of this code?

I also added a new function when relation fork is bigger than the threshold
If (nblocks > BUF_DROP_FULLSCAN_THRESHOLD)
(DropRelFileNodeBuffersOfFork) Perhaps there's a better name for that
function.
However, as expected in the previous discussions, this is a bit slower than the
standard buffer invalidation process, because the whole shared buffers are
scanned nfork times.
Currently, I set the threshold to (NBuffers / 500)

I made a mistake in the v12. I replaced the firstDelBlock[fork_num] with firstDelBlock[block_num],
In the for-loop code block of block_num, because we want to process the current block of per-block loop
OTOH, I used the firstDelBlock[fork_num] when relation fork is bigger than the threshold,
or if the cached blocks of small relations were already invalidated.

Really? I believe that firstDelBlock is an array has only nforks elements.

The logic could be either correct or wrong, so I'd appreciate feedback and comments/advice.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#55Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Kyotaro Horiguchi (#54)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Wed, 16 Sep 2020 11:56:29 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in
(Oops! Some of my comments duplicate with Tsunakawa-san, sorry.)

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#56Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#52)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Sep 16, 2020 at 7:46 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Wed, 2 Sep 2020 08:18:06 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Wed, Sep 2, 2020 at 7:01 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Isn't a relation always locked asscess-exclusively, at truncation
time? If so, isn't even the result of lseek reliable enough?

Even if the relation is locked, background processes like checkpointer
can still touch the relation which might cause problems. Consider a
case where we extend the relation but didn't flush the newly added
pages. Now during truncate operation, checkpointer can still flush
those pages which can cause trouble for truncate. But, I think in the
recovery path such cases won't cause a problem.

I reconsided on this and still have a doubt.

Is this means lseek(SEEK_END) doesn't count blocks that are
write(2)'ed (by smgrextend) but not yet flushed? (I don't think so,
for clarity.) The nblocks cache is added just to reduce the number of
lseek()s and expected to always have the same value with what lseek()
is expected to return.

See comments in ReadBuffer_common() which indicates such a possibility
("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."). Also, refer my previous
email [1]/messages/by-id/CAA4eK1LH2uQWznwtonD+nch76kqzemdTQAnfB06z_LXa6NTFtQ@mail.gmail.com on this and another email link in that email which has a
discussion on this point.

The reason it is reliable only during recovery
is that the cache is not shared but the startup process is the only
process that changes the relation size during recovery.

Yes, that is why we are planning to do this optimization for recovery path.

If any other process can extend the relation while smgrtruncate is
running, the current DropRelFileNodeBuffers should have the chance
that a new buffer for extended area is allocated at a buffer location
where the function already have passed by, which is a disaster.

The relation might have extended before smgrtruncate but the newly
added pages can be flushed by checkpointer during smgrtruncate.

[1]: /messages/by-id/CAA4eK1LH2uQWznwtonD+nch76kqzemdTQAnfB06z_LXa6NTFtQ@mail.gmail.com

--
With Regards,
Amit Kapila.

#57Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#56)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Wed, 16 Sep 2020 08:33:06 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Wed, Sep 16, 2020 at 7:46 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Is this means lseek(SEEK_END) doesn't count blocks that are
write(2)'ed (by smgrextend) but not yet flushed? (I don't think so,
for clarity.) The nblocks cache is added just to reduce the number of
lseek()s and expected to always have the same value with what lseek()
is expected to return.

See comments in ReadBuffer_common() which indicates such a possibility
("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."). Also, refer my previous
email [1] on this and another email link in that email which has a
discussion on this point.

The reason it is reliable only during recovery
is that the cache is not shared but the startup process is the only
process that changes the relation size during recovery.

Yes, that is why we are planning to do this optimization for recovery path.

If any other process can extend the relation while smgrtruncate is
running, the current DropRelFileNodeBuffers should have the chance
that a new buffer for extended area is allocated at a buffer location
where the function already have passed by, which is a disaster.

The relation might have extended before smgrtruncate but the newly
added pages can be flushed by checkpointer during smgrtruncate.

[1] - /messages/by-id/CAA4eK1LH2uQWznwtonD+nch76kqzemdTQAnfB06z_LXa6NTFtQ@mail.gmail.com

Ah! I understood that! The reason we can rely on the cahce is that the
cached value is *not* what lseek returned but how far we intended to
extend. Thank you for the explanation.

By the way I'm not sure that actually happens, but if one smgrextend
call exnteded the relation by two or more blocks, the cache is
invalidated and succeeding smgrnblocks returns lseek()'s result. Don't
we need to guarantee the cache to be valid while recovery?

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#58Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#57)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Sep 16, 2020 at 9:02 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Wed, 16 Sep 2020 08:33:06 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Wed, Sep 16, 2020 at 7:46 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Is this means lseek(SEEK_END) doesn't count blocks that are
write(2)'ed (by smgrextend) but not yet flushed? (I don't think so,
for clarity.) The nblocks cache is added just to reduce the number of
lseek()s and expected to always have the same value with what lseek()
is expected to return.

See comments in ReadBuffer_common() which indicates such a possibility
("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."). Also, refer my previous
email [1] on this and another email link in that email which has a
discussion on this point.

The reason it is reliable only during recovery
is that the cache is not shared but the startup process is the only
process that changes the relation size during recovery.

Yes, that is why we are planning to do this optimization for recovery path.

If any other process can extend the relation while smgrtruncate is
running, the current DropRelFileNodeBuffers should have the chance
that a new buffer for extended area is allocated at a buffer location
where the function already have passed by, which is a disaster.

The relation might have extended before smgrtruncate but the newly
added pages can be flushed by checkpointer during smgrtruncate.

[1] - /messages/by-id/CAA4eK1LH2uQWznwtonD+nch76kqzemdTQAnfB06z_LXa6NTFtQ@mail.gmail.com

Ah! I understood that! The reason we can rely on the cahce is that the
cached value is *not* what lseek returned but how far we intended to
extend. Thank you for the explanation.

By the way I'm not sure that actually happens, but if one smgrextend
call exnteded the relation by two or more blocks, the cache is
invalidated and succeeding smgrnblocks returns lseek()'s result.

Can you think of any such case? I think in recovery we use
XLogReadBufferExtended->ReadBufferWithoutRelcache for reading the page
which seems to be extending page-by-page but there could be some case
where that is not true. One idea is to run regressions and add an
Assert to see if we are extending more than a block during recovery.

Don't
we need to guarantee the cache to be valid while recovery?

One possibility could be that we somehow detect that the value we are
using is cached one and if so then only do this optimization.

--
With Regards,
Amit Kapila.

#59Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#58)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Wed, 16 Sep 2020 10:05:32 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Wed, Sep 16, 2020 at 9:02 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Wed, 16 Sep 2020 08:33:06 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Wed, Sep 16, 2020 at 7:46 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

By the way I'm not sure that actually happens, but if one smgrextend
call exnteded the relation by two or more blocks, the cache is
invalidated and succeeding smgrnblocks returns lseek()'s result.

Can you think of any such case? I think in recovery we use
XLogReadBufferExtended->ReadBufferWithoutRelcache for reading the page
which seems to be extending page-by-page but there could be some case
where that is not true. One idea is to run regressions and add an
Assert to see if we are extending more than a block during recovery.

I agree with you. Actually XLogReadBufferExtended is the only point to
read a page while recovery and seems calling ReadBufferWithoutRelcache
page by page up to the target page. The only case I found where the
cache is invalidated is ALTER TABLE SET TABLESPACE while
wal_level=minimal and not during recovery. smgrextend is called
without smgrnblocks called at the time.

Considering that the behavior of lseek can be a problem only just after
extending a file, an assertion in smgrextend seems to be
enough. Although, I'm not confident on the diagnosis.

--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that nblocks cache
+		 * won't be invalidated by file extension while recoverying.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }

Don't
we need to guarantee the cache to be valid while recovery?

One possibility could be that we somehow detect that the value we are
using is cached one and if so then only do this optimization.

I basically like this direction. But I'm not sure the additional
parameter for smgrnblocks is acceptable.

But on the contrary, it might be a better design that
DropRelFileNodeBuffers gives up the optimization when
smgrnblocks(,,must_accurate = true) returns InvalidBlockNumber.

@@ -544,9 +551,12 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 /*
  *	smgrnblocks() -- Calculate the number of blocks in the
  *					 supplied relation.
+ *
+ *	Returns InvalidBlockNumber if must_accurate is true and smgr_cached_nblocks
+ *	is not available.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool must_accurate)
 {
 	BlockNumber result;

@@ -561,6 +571,17 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)

reln->smgr_cached_nblocks[forknum] = result;

+	/*
+	 * We cannot believe the result from smgr_nblocks is always accurate
+	 * because lseek of buggy Linux kernels doesn't account for a recent
+	 * write. However, we can rely on the result from lseek while recovering
+	 * because the first call to this function is not happen just after a file
+	 * extension. Return values on subsequent calls return cached nblocks,
+	 * which should be accurate during recovery.
+	 */
+	if (!InRecovery && must_accurate)
+		return InvalidBlockNumber;
+
 	return result;
 }

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#60Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#59)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Sep 16, 2020 at 2:02 PM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Wed, 16 Sep 2020 10:05:32 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Wed, Sep 16, 2020 at 9:02 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Wed, 16 Sep 2020 08:33:06 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Wed, Sep 16, 2020 at 7:46 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

By the way I'm not sure that actually happens, but if one smgrextend
call exnteded the relation by two or more blocks, the cache is
invalidated and succeeding smgrnblocks returns lseek()'s result.

Can you think of any such case? I think in recovery we use
XLogReadBufferExtended->ReadBufferWithoutRelcache for reading the page
which seems to be extending page-by-page but there could be some case
where that is not true. One idea is to run regressions and add an
Assert to see if we are extending more than a block during recovery.

I agree with you. Actually XLogReadBufferExtended is the only point to
read a page while recovery and seems calling ReadBufferWithoutRelcache
page by page up to the target page. The only case I found where the
cache is invalidated is ALTER TABLE SET TABLESPACE while
wal_level=minimal and not during recovery. smgrextend is called
without smgrnblocks called at the time.

Considering that the behavior of lseek can be a problem only just after
extending a file, an assertion in smgrextend seems to be
enough. Although, I'm not confident on the diagnosis.

--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
if (reln->smgr_cached_nblocks[forknum] == blocknum)
reln->smgr_cached_nblocks[forknum] = blocknum + 1;
else
+       {
+               /*
+                * DropRelFileNodeBuffers relies on the behavior that nblocks cache
+                * won't be invalidated by file extension while recoverying.
+                */
+               Assert(!InRecovery);
reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+       }
}

Yeah, I have something like this in mind. I am not very sure at this
stage that we want to commit this but for verification purpose,
running regressions it is a good idea.

Don't
we need to guarantee the cache to be valid while recovery?

One possibility could be that we somehow detect that the value we are
using is cached one and if so then only do this optimization.

I basically like this direction. But I'm not sure the additional
parameter for smgrnblocks is acceptable.

But on the contrary, it might be a better design that
DropRelFileNodeBuffers gives up the optimization when
smgrnblocks(,,must_accurate = true) returns InvalidBlockNumber.

I haven't thought about what is the best way to achieve this. Let us
see if Tsunakawa-San or Kirk-San has other ideas on this?

--
With Regards,
Amit Kapila.

#61k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Kyotaro Horiguchi (#59)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wednesday, September 16, 2020 5:32 PM, Kyotaro Horiguchi wrote:

At Wed, 16 Sep 2020 10:05:32 +0530, Amit Kapila <amit.kapila16@gmail.com>
wrote in

On Wed, Sep 16, 2020 at 9:02 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Wed, 16 Sep 2020 08:33:06 +0530, Amit Kapila
<amit.kapila16@gmail.com> wrote in

On Wed, Sep 16, 2020 at 7:46 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

By the way I'm not sure that actually happens, but if one smgrextend
call exnteded the relation by two or more blocks, the cache is
invalidated and succeeding smgrnblocks returns lseek()'s result.

Can you think of any such case? I think in recovery we use
XLogReadBufferExtended->ReadBufferWithoutRelcache for reading the

page

which seems to be extending page-by-page but there could be some case
where that is not true. One idea is to run regressions and add an
Assert to see if we are extending more than a block during recovery.

I agree with you. Actually XLogReadBufferExtended is the only point to read a
page while recovery and seems calling ReadBufferWithoutRelcache page by
page up to the target page. The only case I found where the cache is
invalidated is ALTER TABLE SET TABLESPACE while wal_level=minimal and
not during recovery. smgrextend is called without smgrnblocks called at the
time.

Considering that the behavior of lseek can be a problem only just after
extending a file, an assertion in smgrextend seems to be enough. Although,
I'm not confident on the diagnosis.

--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum,
BlockNumber blocknum,
if (reln->smgr_cached_nblocks[forknum] == blocknum)
reln->smgr_cached_nblocks[forknum] = blocknum + 1;
else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that
nblocks cache
+		 * won't be invalidated by file extension while recoverying.
+		 */
+		Assert(!InRecovery);
reln->smgr_cached_nblocks[forknum] =
InvalidBlockNumber;
+	}
}

Don't
we need to guarantee the cache to be valid while recovery?

One possibility could be that we somehow detect that the value we are
using is cached one and if so then only do this optimization.

I basically like this direction. But I'm not sure the additional parameter for
smgrnblocks is acceptable.

But on the contrary, it might be a better design that DropRelFileNodeBuffers
gives up the optimization when smgrnblocks(,,must_accurate = true) returns
InvalidBlockNumber.

Thank you for your thoughtful reviews and discussions Horiguchi-san, Tsunakawa-san and Amit-san.
Apologies for my carelessness. I've addressed the bugs in the previous version.
1. Getting the total number of blocks for all the specified forks
2. Hashtable probing conditions

I added the suggestion of putting an assert on smgrextend for the XLogReadBufferExtended case,
and I thought that would be enough. I think modifying the smgrnblocks with the addition of new
parameter would complicate the source code because a number of functions call it.
So I thought that maybe putting BlockNumberIsValid(nblocks) in the condition would suffice.
Else, we do full scan of buffer pool.

+                       if ((nblocks / (uint32)NBuffers) < BUF_DROP_FULLSCAN_THRESHOLD &&
+                               BlockNumberIsValid(nblocks))
+                       else
+                       {
				//full scan

Attached is the v14 of the patch. It compiles and passes the tests.
Hoping for your continuous reviews and feedback. Thank you very much.

Regards,
Kirk Jamison

Attachments:

v14-Speedup-dropping-of-relation-buffers-during-recovery.patchapplication/octet-stream; name=v14-Speedup-dropping-of-relation-buffers-during-recovery.patchDownload
From e59bb976b1804ce6f2a2436b176deb3309bf49ce Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 11 Sep 2020 13:00:33 +0000
Subject: [PATCH] Speedup dropping of relation buffers during recovery

---
 src/backend/storage/buffer/bufmgr.c | 126 ++++++++++++++++++++++++++++++++++--
 src/backend/storage/smgr/smgr.c     |   9 ++-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 128 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index a2a963b..6494f9f 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULLSCAN_THRESHOLD		(uint32)(NBuffers / 500)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,8 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+										 BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2972,18 +2976,27 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		XXX The relation might have extended before this, so this path is
+ *		only optimized during recovery when we can get a reliable cached
+ *		value of blocks for specified relation.  See comment in
+ *		smgrnblocks() in smgr.c.  In addition, it is safe to do this since
+ *		there are no other processes but the startup process that changes
+ *		the relation size during recovery. Otherwise, or if not in recovery,
+ *		proceed to usual invalidation process, where it sequentially
+ *		searches the buffer pool.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2997,10 +3010,78 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	if (InRecovery)
+	{
+		BlockNumber		nblocks = 0;
+
+		/* Get the total number of blocks for the supplied relation's fork */
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		block = smgrnblocks(smgr_reln, forkNum[j]);
+			nblocks += block;
+		}
+
+		for (j = 0; j < nforks; j++)
+		{
+			/*
+			 * Do explicit hashtable probe iff the ratio of total number of buffers to be
+			 * truncated against NBuffers is less than the threshold for full-scanning of
+			 * buffer pool. IOW, relation is small enough for its buffers to be removed.
+			 */
+			if ((nblocks / (uint32)NBuffers) < BUF_DROP_FULLSCAN_THRESHOLD &&
+				BlockNumberIsValid(nblocks))
+			{
+				BlockNumber		cur_blk;
+
+				for (cur_blk = firstDelBlock[j]; cur_blk < nblocks; cur_blk++)
+				{
+					uint32		newHash;		/* hash value for newTag */
+					BufferTag	newTag;			/* identity of requested block */
+					LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+					int		buf_id;
+
+					/* create a tag so we can lookup the buffer */
+					INIT_BUFFERTAG(newTag, rnode.node, forkNum[j], cur_blk);
+
+					/* determine its hash code and partition lock ID */
+					newHash = BufTableHashCode(&newTag);
+					newPartitionLock = BufMappingPartitionLock(newHash);
+
+					/* Check that it is in the buffer pool. If not, do nothing */
+					LWLockAcquire(newPartitionLock, LW_SHARED);
+					buf_id = BufTableLookup(&newTag, newHash);
+					LWLockRelease(newPartitionLock);
+
+					if (buf_id < 0)
+						continue;
+
+					bufHdr = GetBufferDescriptor(buf_id);
+
+					buf_state = LockBufHdr(bufHdr);
+
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[j] &&
+						bufHdr->tag.blockNum == cur_blk)
+						InvalidateBuffer(bufHdr);	/* releases spinlock */
+					else
+						UnlockBufHdr(bufHdr, buf_state);
+				}
+			}
+			else
+			{
+				/*
+				 * Relation is larger than the threshold. Execute full scan of
+				 * buffer pool for each fork.
+				 */
+				DropRelFileNodeBuffersOfFork(rnode.node, forkNum[j],
+											 firstDelBlock[j]);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3038,6 +3119,37 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 	}
 }
 
+
+/* -----------------------------------------------------------------
+ *		DropRelFileNodeBuffersOfFork
+ *
+ *		This function removes from the buffer pool the pages for
+ *		the specified relation's fork.
+ * -----------------------------------------------------------------
+ */
+static void
+DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+							 BlockNumber firstDelBlock)
+{
+	int			i;
+
+	for (i = 0; i < NBuffers; i++)
+	{
+		BufferDesc	*bufHdr = GetBufferDescriptor(i);
+		uint32		buf_state;
+
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+
 /* ---------------------------------------------------------------------
  *		DropRelFileNodesAllBuffers
  *
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..450819f 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension while recovering.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
@@ -583,7 +590,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

#62tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#61)
RE: [Patch] Optimize dropping of relation buffers using dlist

I looked at v14.

(1)
+		/* Get the total number of blocks for the supplied relation's fork */
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		block = smgrnblocks(smgr_reln, forkNum[j]);
+			nblocks += block;
+		}

Why do you sum all forks?

(2)
+			if ((nblocks / (uint32)NBuffers) < BUF_DROP_FULLSCAN_THRESHOLD &&
+				BlockNumberIsValid(nblocks))
+			{

The division by NBuffers is not necessary, because both sides of = are number of blocks.
Why is BlockNumberIsValid(nblocks)) call needed?

(3)
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension while recovering.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}

I think this change is not directly related to this patch and can be a separate patch, but I want to leave the decision up to a committer.

Regards
Takayuki Tsunakawa

#63tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Amit Kapila (#60)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Amit Kapila <amit.kapila16@gmail.com>

Don't
we need to guarantee the cache to be valid while recovery?

One possibility could be that we somehow detect that the value we
are using is cached one and if so then only do this optimization.

I basically like this direction. But I'm not sure the additional
parameter for smgrnblocks is acceptable.

But on the contrary, it might be a better design that
DropRelFileNodeBuffers gives up the optimization when
smgrnblocks(,,must_accurate = true) returns InvalidBlockNumber.

I haven't thought about what is the best way to achieve this. Let us see if
Tsunakawa-San or Kirk-San has other ideas on this?

I see no need for smgrnblocks() to add an argument as it returns the correct cached or measured value.

Regards
Takayuki Tsunakawa

#64k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#62)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wednesday, September 23, 2020 11:26 AM, Tsunakawa, Takayuki wrote:

I looked at v14.

Thank you for checking it!

(1)
+		/* Get the total number of blocks for the supplied relation's
fork */
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		block =
smgrnblocks(smgr_reln, forkNum[j]);
+			nblocks += block;
+		}

Why do you sum all forks?

I revised the patch based from my understanding of Horiguchi-san's comment,
but I could be wrong.
Quoting:

" 
+			/* Get the number of blocks for the supplied relation's fork */
+			nblocks = smgrnblocks(smgr_reln, forkNum[fork_num]);
+			Assert(BlockNumberIsValid(nblocks));
+
+			if (nblocks < BUF_DROP_FULLSCAN_THRESHOLD)

As mentioned upthread, the criteria whether we do full-scan or
lookup-drop is how large portion of NBUFFERS this relation-drop can be
going to invalidate. So the nblocks above should be the sum of number
of blocks to be truncated (not just the total number of blocks) of all
designated forks. Then once we decided to do lookup-drop method, we
do that for all forks."

(2)
+			if ((nblocks / (uint32)NBuffers) <
BUF_DROP_FULLSCAN_THRESHOLD &&
+				BlockNumberIsValid(nblocks))
+			{

The division by NBuffers is not necessary, because both sides of = are
number of blocks.

Again I based it from my understanding of the comment above,
so nblocks is the sum of all blocks to be truncated for all forks.

Why is BlockNumberIsValid(nblocks)) call needed?

I thought we need to ensure that nblocks is not invalid, so I also added

(3)
if (reln->smgr_cached_nblocks[forknum] == blocknum)
reln->smgr_cached_nblocks[forknum] = blocknum + 1;
else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that
cached nblocks
+		 * won't be invalidated by file extension while recovering.
+		 */
+		Assert(!InRecovery);
reln->smgr_cached_nblocks[forknum] =
InvalidBlockNumber;
+	}

I think this change is not directly related to this patch and can be a separate
patch, but I want to leave the decision up to a committer.

This is noted. Once we clarified the above comments, I'll put it in a separate patch if it's necessary,

Thank you very much for the reviews.

Best regards,
Kirk Jamison

#65Amit Kapila
amit.kapila16@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#62)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Sep 23, 2020 at 7:56 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

(3)
if (reln->smgr_cached_nblocks[forknum] == blocknum)
reln->smgr_cached_nblocks[forknum] = blocknum + 1;
else
+       {
+               /*
+                * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+                * won't be invalidated by file extension while recovering.
+                */
+               Assert(!InRecovery);
reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+       }

I think this change is not directly related to this patch and can be a separate patch, but I want to leave the decision up to a committer.

We have added this mainly for testing purpose, basically this
assertion should not fail during the regression tests. We can keep it
in a separate patch but need to ensure that. If this fails then we
can't rely on the caching behaviour during recovery which is actually
required for the correctness of patch.

--
With Regards,
Amit Kapila.

#66Amit Kapila
amit.kapila16@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#63)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Sep 23, 2020 at 8:04 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Amit Kapila <amit.kapila16@gmail.com>

Don't
we need to guarantee the cache to be valid while recovery?

One possibility could be that we somehow detect that the value we
are using is cached one and if so then only do this optimization.

I basically like this direction. But I'm not sure the additional
parameter for smgrnblocks is acceptable.

But on the contrary, it might be a better design that
DropRelFileNodeBuffers gives up the optimization when
smgrnblocks(,,must_accurate = true) returns InvalidBlockNumber.

I haven't thought about what is the best way to achieve this. Let us see if
Tsunakawa-San or Kirk-San has other ideas on this?

I see no need for smgrnblocks() to add an argument as it returns the correct cached or measured value.

The idea is that we can't use this optimization if the value is not
cached because we can't rely on lseek behavior. See all the discussion
between Horiguchi-San and me in the thread above. So, how would you
ensure that if we don't use Kirk-San's proposal?

--
With Regards,
Amit Kapila.

#67tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#64)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

I revised the patch based from my understanding of Horiguchi-san's comment,
but I could be wrong.
Quoting:

"
+			/* Get the number of blocks for the supplied relation's
fork */
+			nblocks = smgrnblocks(smgr_reln,
forkNum[fork_num]);
+			Assert(BlockNumberIsValid(nblocks));
+
+			if (nblocks < BUF_DROP_FULLSCAN_THRESHOLD)

As mentioned upthread, the criteria whether we do full-scan or
lookup-drop is how large portion of NBUFFERS this relation-drop can be
going to invalidate. So the nblocks above should be the sum of number
of blocks to be truncated (not just the total number of blocks) of all
designated forks. Then once we decided to do lookup-drop method, we
do that for all forks."

One takeaway from Horiguchi-san's comment is to use the number of blocks to invalidate for comparison, instead of all blocks in the fork. That is, use

nblocks = smgrnblocks(fork) - firstDelBlock[fork];

Does this make sense?

What do you think is the reason for summing up all forks? I didn't understand why. Typically, FSM and VM forks are very small. If the main fork is larger than NBuffers / 500, then v14 scans the entire shared buffers for the FSM and VM forks as well as the main fork, resulting in three scans in total.

Also, if you want to judge the criteria based on the total blocks of all forks, the following if should be placed outside the for loop, right? Because this if condition doesn't change inside the for loop.

+			if ((nblocks / (uint32)NBuffers) < BUF_DROP_FULLSCAN_THRESHOLD &&
+				BlockNumberIsValid(nblocks))
+			{
(2)
+			if ((nblocks / (uint32)NBuffers) <
BUF_DROP_FULLSCAN_THRESHOLD &&
+				BlockNumberIsValid(nblocks))
+			{

The division by NBuffers is not necessary, because both sides of = are
number of blocks.

Again I based it from my understanding of the comment above,
so nblocks is the sum of all blocks to be truncated for all forks.

But the left expression of "<" is a percentage, while the right one is a block count. Two different units are compared.

Why is BlockNumberIsValid(nblocks)) call needed?

I thought we need to ensure that nblocks is not invalid, so I also added

When is it invalid? smgrnblocks() seems to always return a valid block number. Am I seeing a different source code (I saw HEAD)?

Regards
Takayuki Tsunakawa

#68tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Amit Kapila (#66)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Amit Kapila <amit.kapila16@gmail.com>

The idea is that we can't use this optimization if the value is not
cached because we can't rely on lseek behavior. See all the discussion
between Horiguchi-San and me in the thread above. So, how would you
ensure that if we don't use Kirk-San's proposal?

Hmm, buggy Linux kernel... (Until when should we be worried about the bug?)

According to the following Horiguchi-san's suggestion, it's during normal operation, not during recovery, when we should be careful, right? Then, we can use the current smgrnblocks() as is?

+	/*
+	 * We cannot believe the result from smgr_nblocks is always accurate
+	 * because lseek of buggy Linux kernels doesn't account for a recent
+	 * write. However, we can rely on the result from lseek while recovering
+	 * because the first call to this function is not happen just after a file
+	 * extension. Return values on subsequent calls return cached nblocks,
+	 * which should be accurate during recovery.
+	 */
+	if (!InRecovery && must_accurate)
+		return InvalidBlockNumber;
+
 	return result;
} 

If smgrnblocks() could return a smaller value than the actual file size by one block even during recovery, how about always adding one to the return value of smgrnblocks() in DropRelFileNodeBuffers()? When smgrnblocks() actually returned the correct value, the extra one block is not found in the shared buffer, so DropRelFileNodeBuffers() does no harm.

Or, add a new function like smgrnblocks_precise() to avoid adding an argument to smgrnblocks()?

Regards
Takayuki Tsunakawa

#69k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#67)
2 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wednesday, September 23, 2020 2:37 PM, Tsunakawa, Takayuki wrote:

I revised the patch based from my understanding of Horiguchi-san's
comment, but I could be wrong.
Quoting:

"
+ /* Get the number of blocks for the supplied

relation's

fork */
+			nblocks = smgrnblocks(smgr_reln,
forkNum[fork_num]);
+			Assert(BlockNumberIsValid(nblocks));
+
+			if (nblocks <

BUF_DROP_FULLSCAN_THRESHOLD)

As mentioned upthread, the criteria whether we do full-scan or
lookup-drop is how large portion of NBUFFERS this relation-drop can be
going to invalidate. So the nblocks above should be the sum of number
of blocks to be truncated (not just the total number of blocks) of all
designated forks. Then once we decided to do lookup-drop method, we
do that for all forks."

One takeaway from Horiguchi-san's comment is to use the number of blocks
to invalidate for comparison, instead of all blocks in the fork. That is, use

nblocks = smgrnblocks(fork) - firstDelBlock[fork];

Does this make sense?

Hmm. Ok, I think it got too much to my head that I misunderstood what it meant.
I'll debug again by using ereport just to check the values and behavior are correct.
Your comment about V14 patch has dawned on me that it reverted to previous
slower version where we scan NBuffers for each fork. Thank you for explaining it.

What do you think is the reason for summing up all forks? I didn't
understand why. Typically, FSM and VM forks are very small. If the main
fork is larger than NBuffers / 500, then v14 scans the entire shared buffers for
the FSM and VM forks as well as the main fork, resulting in three scans in
total.

Also, if you want to judge the criteria based on the total blocks of all forks, the
following if should be placed outside the for loop, right? Because this if
condition doesn't change inside the for loop.

+			if ((nblocks / (uint32)NBuffers) <
BUF_DROP_FULLSCAN_THRESHOLD &&
+				BlockNumberIsValid(nblocks))
+			{
(2)
+			if ((nblocks / (uint32)NBuffers) <
BUF_DROP_FULLSCAN_THRESHOLD &&
+				BlockNumberIsValid(nblocks))
+			{

The division by NBuffers is not necessary, because both sides of =
are number of blocks.

Again I based it from my understanding of the comment above, so
nblocks is the sum of all blocks to be truncated for all forks.

But the left expression of "<" is a percentage, while the right one is a block
count. Two different units are compared.

Right. Makes sense. Fixed.

Why is BlockNumberIsValid(nblocks)) call needed?

I thought we need to ensure that nblocks is not invalid, so I also
added

When is it invalid? smgrnblocks() seems to always return a valid block
number. Am I seeing a different source code (I saw HEAD)?

It's based from the discussion upthread to guarantee the cache to be valid while recovery
and that we don't want to proceed with the optimization in case that nblocks is invalid.
It may not be needed so I already removed it, because the correct direction is ensuring that
smgrnblocks return the precise value.
Considering the test case that Horiguchi-san suggested (attached as separate patch),
then maybe there's no need to indicate it in the loop condition.
For now, I haven't modified the design (or created a new function) of smgrnblocks,
and I just updated the patches based from the recent comments.

Thank you very much again for the reviews.

Best regards,
Kirk Jamison

Attachments:

v15-Speedup-dropping-of-relation-buffers-during-recovery.patchapplication/octet-stream; name=v15-Speedup-dropping-of-relation-buffers-during-recovery.patchDownload
From 62fab3f5ad7adfa64a0a0732b7983f9ab334a1f0 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 11 Sep 2020 13:00:33 +0000
Subject: [PATCH] Optimize dropping of relation buffers during recovery

---
 src/backend/storage/buffer/bufmgr.c | 120 +++++++++++++++++++++++++++++++++---
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 115 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index a2a963b..11f450b 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULLSCAN_THRESHOLD		(uint32)(NBuffers / 500)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,8 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+										 BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2972,18 +2976,27 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		XXX The relation might have extended before this, so this path is
+ *		only optimized during recovery when we can get a reliable cached
+ *		value of blocks for specified relation.  See comment in
+ *		smgrnblocks() in smgr.c.  In addition, it is safe to do this since
+ *		there are no other processes but the startup process that changes
+ *		the relation size during recovery.  Otherwise, or if not in
+ *		recovery, proceed to usual invalidation process, where it
+ *		sequentially searches the buffer pool.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2997,10 +3010,72 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	if (InRecovery)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			/* Get the total number of blocks for the supplied relation's fork */
+			BlockNumber		nblocks;
+			nblocks = smgrnblocks(smgr_reln, forkNum[j]) - firstDelBlock[j];
+
+			/*
+			 * Do explicit hashtable probe iff the ratio of total number of buffers to be
+			 * truncated against NBuffers is less than the threshold for full-scanning of
+			 * buffer pool. IOW, relation is small enough for its buffers to be removed.
+			 */
+			if (nblocks < BUF_DROP_FULLSCAN_THRESHOLD)
+			{
+				BlockNumber		cur_blk;
+
+				for (cur_blk = firstDelBlock[j]; cur_blk < nblocks; cur_blk++)
+				{
+					uint32		newHash;		/* hash value for newTag */
+					BufferTag	newTag;			/* identity of requested block */
+					LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+					int		buf_id;
+
+					/* create a tag so we can lookup the buffer */
+					INIT_BUFFERTAG(newTag, rnode.node, forkNum[j], cur_blk);
+
+					/* determine its hash code and partition lock ID */
+					newHash = BufTableHashCode(&newTag);
+					newPartitionLock = BufMappingPartitionLock(newHash);
+
+					/* Check that it is in the buffer pool. If not, do nothing */
+					LWLockAcquire(newPartitionLock, LW_SHARED);
+					buf_id = BufTableLookup(&newTag, newHash);
+					LWLockRelease(newPartitionLock);
+
+					if (buf_id < 0)
+						continue;
+
+					bufHdr = GetBufferDescriptor(buf_id);
+
+					buf_state = LockBufHdr(bufHdr);
+
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[j] &&
+						bufHdr->tag.blockNum == cur_blk)
+						InvalidateBuffer(bufHdr);	/* releases spinlock */
+					else
+						UnlockBufHdr(bufHdr, buf_state);
+				}
+			}
+			else
+			{
+				/*
+				 * Relation is larger than the threshold. Execute full scan of
+				 * buffer pool for each fork.
+				 */
+				DropRelFileNodeBuffersOfFork(rnode.node, forkNum[j],
+											 firstDelBlock[j]);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3038,6 +3113,37 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 	}
 }
 
+
+/* -----------------------------------------------------------------
+ *		DropRelFileNodeBuffersOfFork
+ *
+ *		This function removes from the buffer pool the pages for
+ *		the specified relation's fork.
+ * -----------------------------------------------------------------
+ */
+static void
+DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+							 BlockNumber firstDelBlock)
+{
+	int			i;
+
+	for (i = 0; i < NBuffers; i++)
+	{
+		BufferDesc	*bufHdr = GetBufferDescriptor(i);
+		uint32		buf_state;
+
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+
 /* ---------------------------------------------------------------------
  *		DropRelFileNodesAllBuffers
  *
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..5238c6c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -583,7 +583,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v1-Prevent-smgrextend-from-invalidating-blocks-during-recovery.patchapplication/octet-stream; name=v1-Prevent-smgrextend-from-invalidating-blocks-during-recovery.patchDownload
From 7b3fc6096effb862677e868699f33759bf0ff087 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH] Prevent invalidating of cached blocks in smgrextend during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

#70Amit Kapila
amit.kapila16@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#68)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Sep 23, 2020 at 12:00 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Amit Kapila <amit.kapila16@gmail.com>

The idea is that we can't use this optimization if the value is not
cached because we can't rely on lseek behavior. See all the discussion
between Horiguchi-San and me in the thread above. So, how would you
ensure that if we don't use Kirk-San's proposal?

Hmm, buggy Linux kernel... (Until when should we be worried about the bug?)

According to the following Horiguchi-san's suggestion, it's during normal operation, not during recovery, when we should be careful, right?

No, during recovery also we need to be careful. We need to ensure that
we use cached value during recovery and cached value is always
up-to-date. We can't rely on lseek and I have provided some scenario
up thread [1]/messages/by-id/CAA4eK1LqaJvT=bFOpc4i5Haq4oaVQ6wPbAcg64-Kt1qzp_MZYA@mail.gmail.com where such behavior can cause problem and then see the
response from Tom Lane why the same can be true for recovery as well.

The basic approach we are trying to pursue here is to rely on the
cached value of 'number of blocks' (as that always gives correct value
and even if there is a problem that will be our bug, we don't need to
rely on OS for correct value and it will be better w.r.t performance
as well). It is currently only possible during recovery so we are
using it in recovery path and later once Thomas's patch to cache it
for non-recovery cases is also done, we can use it for non-recovery
cases as well.

[1]: /messages/by-id/CAA4eK1LqaJvT=bFOpc4i5Haq4oaVQ6wPbAcg64-Kt1qzp_MZYA@mail.gmail.com

--
With Regards,
Amit Kapila.

#71tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#69)
RE: [Patch] Optimize dropping of relation buffers using dlist

In v15:

(1)
+ for (cur_blk = firstDelBlock[j]; cur_blk < nblocks; cur_blk++)

The right side of "cur_blk <" should not be nblocks, because nblocks is not the number of the relation fork anymore.

(2)
+			BlockNumber		nblocks;
+			nblocks = smgrnblocks(smgr_reln, forkNum[j]) - firstDelBlock[j];

You should either:

* Combine the two lines into one: BlockNumber nblocks = ...;

or

* Put an empty line between the two lines to separate declarations and execution statements.

After correcting these, I think you can check the recovery performance.

Regards
Takayuki Tsunakawa

#72k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#71)
2 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, September 24, 2020 1:27 PM, Tsunakawa-san wrote:

(1)
+ for (cur_blk = firstDelBlock[j]; cur_blk <
nblocks; cur_blk++)

The right side of "cur_blk <" should not be nblocks, because nblocks is not
the number of the relation fork anymore.

Right. Fixed. It should be the total number of (n)blocks of relation.

(2)
+			BlockNumber		nblocks;
+			nblocks = smgrnblocks(smgr_reln, forkNum[j]) -
firstDelBlock[j];

You should either:

* Combine the two lines into one: BlockNumber nblocks = ...;

or

* Put an empty line between the two lines to separate declarations and
execution statements.

Right. I separated them in the updated patch. And to prevent confusion,
instead of nblocks, nTotalBlocks & nBlocksToInvalidate are used.

/* Get the total number of blocks for the supplied relation's fork */
nTotalBlocks = smgrnblocks(smgr_reln, forkNum[j]);

/* Get the total number of blocks to be invalidated for the specified fork */
nBlocksToInvalidate = nTotalBlocks - firstDelBlock[j];

After correcting these, I think you can check the recovery performance.

I'll send performance measurement results in the next email. Thanks a lot for the reviews!

Regards,
Kirk Jamison

Attachments:

v16-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v16-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 4b051a1aae7ad344269ca81322b956e24158a881 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 11 Sep 2020 13:00:33 +0000
Subject: [PATCH] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork by calling smgrnblocks(), and it's safe
because there are no other processes but the startup process that
changes the relation size during recovery.  Otherwise, or if not in
recovery, proceed to sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 125 ++++++++++++++++++++++++++++++++++--
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 120 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index a2a963b..7c2c196 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULLSCAN_THRESHOLD		(uint32)(NBuffers / 500)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,8 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+										 BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2972,18 +2976,27 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		XXX The relation might have extended before this, so this path is
+ *		only optimized during recovery when we can get a reliable cached
+ *		value of blocks for specified relation.  See comment in
+ *		smgrnblocks() in smgr.c.  In addition, it is safe to do this since
+ *		there are no other processes but the startup process that changes
+ *		the relation size during recovery.  Otherwise, or if not in
+ *		recovery, proceed to usual invalidation process, where it
+ *		sequentially searches the buffer pool.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2997,10 +3010,77 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	if (InRecovery)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber	nTotalBlocks; /* total nblocks */
+			BlockNumber nBlocksToInvalidate; /* total nblocks to be invalidated */
+
+			/* Get the total number of blocks for the supplied relation's fork */
+			nTotalBlocks = smgrnblocks(smgr_reln, forkNum[j]);
+
+			/* Get the total number of blocks to be invalidated for the specified fork */
+			nBlocksToInvalidate = nTotalBlocks - firstDelBlock[j];
+
+			/*
+			 * Do explicit hashtable probe iff the ratio of total number of buffers to be
+			 * truncated against NBuffers is less than the threshold for full-scanning of
+			 * buffer pool. IOW, relation is small enough for its buffers to be removed.
+			 */
+			if (nBlocksToInvalidate < BUF_DROP_FULLSCAN_THRESHOLD)
+			{
+				BlockNumber		curBlock;
+
+				for (curBlock = firstDelBlock[j]; curBlock < nTotalBlocks; curBlock++)
+				{
+					uint32		newHash;		/* hash value for newTag */
+					BufferTag	newTag;			/* identity of requested block */
+					LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+					int		buf_id;
+
+					/* create a tag so we can lookup the buffer */
+					INIT_BUFFERTAG(newTag, rnode.node, forkNum[j], curBlock);
+
+					/* determine its hash code and partition lock ID */
+					newHash = BufTableHashCode(&newTag);
+					newPartitionLock = BufMappingPartitionLock(newHash);
+
+					/* Check that it is in the buffer pool. If not, do nothing */
+					LWLockAcquire(newPartitionLock, LW_SHARED);
+					buf_id = BufTableLookup(&newTag, newHash);
+					LWLockRelease(newPartitionLock);
+
+					if (buf_id < 0)
+						continue;
+
+					bufHdr = GetBufferDescriptor(buf_id);
+
+					buf_state = LockBufHdr(bufHdr);
+
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[j] &&
+						bufHdr->tag.blockNum == curBlock)
+						InvalidateBuffer(bufHdr);	/* releases spinlock */
+					else
+						UnlockBufHdr(bufHdr, buf_state);
+				}
+			}
+			else
+			{
+				/*
+				 * Relation is larger than the threshold. Execute full scan of
+				 * buffer pool for each fork.
+				 */
+				DropRelFileNodeBuffersOfFork(rnode.node, forkNum[j],
+											 firstDelBlock[j]);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3038,6 +3118,37 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 	}
 }
 
+
+/* -----------------------------------------------------------------
+ *		DropRelFileNodeBuffersOfFork
+ *
+ *		This function removes from the buffer pool the pages for
+ *		the specified relation's fork.
+ * -----------------------------------------------------------------
+ */
+static void
+DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+							 BlockNumber firstDelBlock)
+{
+	int			i;
+
+	for (i = 0; i < NBuffers; i++)
+	{
+		BufferDesc	*bufHdr = GetBufferDescriptor(i);
+		uint32		buf_state;
+
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+
 /* ---------------------------------------------------------------------
  *		DropRelFileNodesAllBuffers
  *
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..5238c6c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -583,7 +583,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchapplication/octet-stream; name=v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchDownload
From 7b3fc6096effb862677e868699f33759bf0ff087 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH] Prevent invalidating of cached blocks in smgrextend during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

#73Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#67)
Re: [Patch] Optimize dropping of relation buffers using dlist

Hello.

At Wed, 23 Sep 2020 05:37:24 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

# Wow. I'm surprised to read it..

I revised the patch based from my understanding of Horiguchi-san's comment,
but I could be wrong.
Quoting:

"
+			/* Get the number of blocks for the supplied relation's
fork */
+			nblocks = smgrnblocks(smgr_reln,
forkNum[fork_num]);
+			Assert(BlockNumberIsValid(nblocks));
+
+			if (nblocks < BUF_DROP_FULLSCAN_THRESHOLD)

As mentioned upthread, the criteria whether we do full-scan or
lookup-drop is how large portion of NBUFFERS this relation-drop can be
going to invalidate. So the nblocks above should be the sum of number
of blocks to be truncated (not just the total number of blocks) of all
designated forks. Then once we decided to do lookup-drop method, we
do that for all forks."

One takeaway from Horiguchi-san's comment is to use the number of blocks to invalidate for comparison, instead of all blocks in the fork. That is, use

nblocks = smgrnblocks(fork) - firstDelBlock[fork];

Does this make sense?

What do you think is the reason for summing up all forks? I didn't understand why. Typically, FSM and VM forks are very small. If the main fork is larger than NBuffers / 500, then v14 scans the entire shared buffers for the FSM and VM forks as well as the main fork, resulting in three scans in total.

I thought of summing up smgrnblocks(fork) - firstDelBlock[fork] of all
folks. I don't mind omitting non-main forks but a comment to explain
the reason or reasoning would be needed.

reards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#74k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#72)
2 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi.

I'll send performance measurement results in the next email. Thanks a lot for
the reviews!

Below are the performance measurement results.
I was only able to use low-spec machine:
CPU 4v, Memory 8GB, RHEL, xfs filesystem.

[Failover/Recovery Test]
1. (Master) Create table (ex. 10,000 tables). Insert data to tables.
2. (M) DELETE FROM TABLE (ex. all rows of 10,000 tables)
3. (Standby) To test with failover, pause the WAL replay on standby server.
(SELECT pg_wal_replay_pause();)
4. (M) psql -c "\timing on" (measures total execution of SQL queries)
5. (M) VACUUM (whole db)
6. (M) After vacuum finishes, stop primary server: pg_ctl stop -w -mi
7. (S) Resume wal replay and promote standby.
Because it's difficult to measure recovery time I used the attached script (resume.sh)
that prints timestamp before and after promotion. It basically does the following
- "SELECT pg_wal_replay_resume();" is executed and the WAL application is resumed.
- "pg_ctl promote" to promote standby.
- The time difference of "select pg_is_in_recovery();" from "t" to "f" is measured.

[Results]
Recovery/Failover performance (in seconds). 3 trial runs.

| shared_buffers | master | patch | %reg |
|----------------|--------|--------|---------|
| 128MB | 32.406 | 33.785 | 4.08% |
| 1GB | 36.188 | 32.747 | -10.51% |
| 2GB | 41.996 | 32.88 | -27.73% |

There's a bit of small regression with the default shared_buffers (128MB),
but as for the recovery time when we have large NBuffers, it's now at least almost constant
so there's boosted performance. IOW, we enter the optimization most of the time
during recovery.

I also did similar benchmark performance as what Tomas did [1]/messages/by-id/20200806213334.3bzadeirly3mdtzl@development,
simple "pgbench -S" tests (warmup and then 15 x 1-minute runs with
1, 8 and 16 clients, but I'm not sure if my machine is reliable enough to
produce reliable results for 8 clients and more.

| # | master | patch | %reg |
|------------|-------------|-------------|--------|
| 1 client | 1676.937825 | 1707.018029 | -1.79% |
| 8 clients | 7706.835401 | 7529.089044 | 2.31% |
| 16 clients | 9823.65254 | 9991.184206 | -1.71% |

If there's additional/necessary performance measurement, kindly advise me too.
Thank you in advance.

[1]: /messages/by-id/20200806213334.3bzadeirly3mdtzl@development

Best regards,
Kirk Jamison

Attachments:

resume.shapplication/octet-stream; name=resume.shDownload
run.shapplication/octet-stream; name=run.shDownload
#75tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Amit Kapila (#70)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Amit Kapila <amit.kapila16@gmail.com>

No, during recovery also we need to be careful. We need to ensure that
we use cached value during recovery and cached value is always
up-to-date. We can't rely on lseek and I have provided some scenario
up thread [1] where such behavior can cause problem and then see the
response from Tom Lane why the same can be true for recovery as well.

The basic approach we are trying to pursue here is to rely on the
cached value of 'number of blocks' (as that always gives correct value
and even if there is a problem that will be our bug, we don't need to
rely on OS for correct value and it will be better w.r.t performance
as well). It is currently only possible during recovery so we are
using it in recovery path and later once Thomas's patch to cache it
for non-recovery cases is also done, we can use it for non-recovery
cases as well.

Although I may be still confused, I understood that Kirk-san's patch should:

* Still focus on speeding up the replay of TRUNCATE during recovery.

* During recovery, DropRelFileNodeBuffers() gets the cached size of the relation fork. If it is cached, trust it and optimize the buffer invalidation. If it's not cached, we can't trust the return value of smgrnblocks() because it's the lseek(END) return value, so we avoid the optimization.

* Then, add a new function, say, smgrnblocks_cached() that simply returns the cached block count, and DropRelFileNodeBuffers() uses it instead of smgrnblocks().

Regards
Takayuki Tsunakawa

#76tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#74)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

[Results]
Recovery/Failover performance (in seconds). 3 trial runs.

| shared_buffers | master | patch | %reg |
|----------------|--------|--------|---------|
| 128MB | 32.406 | 33.785 | 4.08% |
| 1GB | 36.188 | 32.747 | -10.51% |
| 2GB | 41.996 | 32.88 | -27.73% |

Thanks for sharing good results. We want to know if we can get as significant results as you gained before with hundreds of GBs of shared buffers, don't we?

I also did similar benchmark performance as what Tomas did [1], simple
"pgbench -S" tests (warmup and then 15 x 1-minute runs with 1, 8 and 16
clients, but I'm not sure if my machine is reliable enough to produce reliable
results for 8 clients and more.

Let me confirm just in case. Your patch should not affect pgbench performance, but you measured it. Is there anything you're concerned about?

Regards
Takayuki Tsunakawa

#77k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#76)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Friday, September 25, 2020 6:02 PM, Tsunakawa-san wrote:

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

[Results]
Recovery/Failover performance (in seconds). 3 trial runs.

| shared_buffers | master | patch | %reg |
|----------------|--------|--------|---------|
| 128MB | 32.406 | 33.785 | 4.08% |
| 1GB | 36.188 | 32.747 | -10.51% |
| 2GB | 41.996 | 32.88 | -27.73% |

Thanks for sharing good results. We want to know if we can get as
significant results as you gained before with hundreds of GBs of shared
buffers, don't we?

Yes. But I don't have a high-spec machine I could use at the moment.
I'll try if I can get one by next week. Or if someone would like to reproduce the
test with their available higher spec machines, it'd would be much appreciated.
The test case is upthread [1]/messages/by-id/OSBPR01MB2341683DEDE0E7A8D045036FEF360@OSBPR01MB2341.jpnprd01.prod.outlook.com

I also did similar benchmark performance as what Tomas did [1], simple
"pgbench -S" tests (warmup and then 15 x 1-minute runs with 1, 8 and
16 clients, but I'm not sure if my machine is reliable enough to
produce reliable results for 8 clients and more.

Let me confirm just in case. Your patch should not affect pgbench
performance, but you measured it. Is there anything you're concerned
about?

Not really. Because In the previous emails, the argument was the BufferAlloc
overhead. But we don't have it in the latest patch. But just in case somebody
asks about benchmark performance, I also posted the results.

[1]: /messages/by-id/OSBPR01MB2341683DEDE0E7A8D045036FEF360@OSBPR01MB2341.jpnprd01.prod.outlook.com

Regards,
Kirk Jamison

#78Amit Kapila
amit.kapila16@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#75)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Sep 25, 2020 at 2:25 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Amit Kapila <amit.kapila16@gmail.com>

No, during recovery also we need to be careful. We need to ensure that
we use cached value during recovery and cached value is always
up-to-date. We can't rely on lseek and I have provided some scenario
up thread [1] where such behavior can cause problem and then see the
response from Tom Lane why the same can be true for recovery as well.

The basic approach we are trying to pursue here is to rely on the
cached value of 'number of blocks' (as that always gives correct value
and even if there is a problem that will be our bug, we don't need to
rely on OS for correct value and it will be better w.r.t performance
as well). It is currently only possible during recovery so we are
using it in recovery path and later once Thomas's patch to cache it
for non-recovery cases is also done, we can use it for non-recovery
cases as well.

Although I may be still confused, I understood that Kirk-san's patch should:

* Still focus on speeding up the replay of TRUNCATE during recovery.

* During recovery, DropRelFileNodeBuffers() gets the cached size of the relation fork. If it is cached, trust it and optimize the buffer invalidation. If it's not cached, we can't trust the return value of smgrnblocks() because it's the lseek(END) return value, so we avoid the optimization.

I agree with the above two points.

* Then, add a new function, say, smgrnblocks_cached() that simply returns the cached block count, and DropRelFileNodeBuffers() uses it instead of smgrnblocks().

I am not sure if it worth adding a new function for this. Why not
simply add a boolean variable in smgrnblocks for this? BTW, AFAICS,
the latest patch doesn't have code to address this point.

--
With Regards,
Amit Kapila.

#79Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#74)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Sep 25, 2020 at 1:49 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Hi.

I'll send performance measurement results in the next email. Thanks a lot for
the reviews!

Below are the performance measurement results.
I was only able to use low-spec machine:
CPU 4v, Memory 8GB, RHEL, xfs filesystem.

[Failover/Recovery Test]
1. (Master) Create table (ex. 10,000 tables). Insert data to tables.
2. (M) DELETE FROM TABLE (ex. all rows of 10,000 tables)
3. (Standby) To test with failover, pause the WAL replay on standby server.
(SELECT pg_wal_replay_pause();)
4. (M) psql -c "\timing on" (measures total execution of SQL queries)
5. (M) VACUUM (whole db)
6. (M) After vacuum finishes, stop primary server: pg_ctl stop -w -mi
7. (S) Resume wal replay and promote standby.
Because it's difficult to measure recovery time I used the attached script (resume.sh)
that prints timestamp before and after promotion. It basically does the following
- "SELECT pg_wal_replay_resume();" is executed and the WAL application is resumed.
- "pg_ctl promote" to promote standby.
- The time difference of "select pg_is_in_recovery();" from "t" to "f" is measured.

[Results]
Recovery/Failover performance (in seconds). 3 trial runs.

| shared_buffers | master | patch | %reg |
|----------------|--------|--------|---------|
| 128MB | 32.406 | 33.785 | 4.08% |
| 1GB | 36.188 | 32.747 | -10.51% |
| 2GB | 41.996 | 32.88 | -27.73% |

There's a bit of small regression with the default shared_buffers (128MB),

I feel we should try to address this. Basically, we can see the
smallest value of shared buffers above which the new algorithm is
beneficial and try to use that as threshold for doing this
optimization. I don't think it is beneficial to use this optimization
for a small value of shared_buffers.

but as for the recovery time when we have large NBuffers, it's now at least almost constant
so there's boosted performance. IOW, we enter the optimization most of the time
during recovery.

Yeah, that is good to see. We can probably try to check with a much
larger value of shared buffers.

--
With Regards,
Amit Kapila.

#80tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Amit Kapila (#78)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Amit Kapila <amit.kapila16@gmail.com>

I agree with the above two points.

Thank you. I'm relieved to know I didn't misunderstand.

* Then, add a new function, say, smgrnblocks_cached() that simply returns

the cached block count, and DropRelFileNodeBuffers() uses it instead of
smgrnblocks().

I am not sure if it worth adding a new function for this. Why not simply add a
boolean variable in smgrnblocks for this?

One reason is that adding an argument requires modification of existing call sites (10 + a few). Another is that, although this may be different for each person's taste, it's sometimes not easy to understand when a function call with true/false appears. One such example is find_XXX(some_args, true/false), where the true/false represents missing_ok. Another example is as follows. I often wonder "what's the meaning of this false, and that true?"

if (!InstallXLogFileSegment(&destsegno, tmppath, false, 0, false))
elog(ERROR, "InstallXLogFileSegment should not have failed");

Fortunately, the new function is very short and doesn't duplicate much code. The function is a simple getter and the function name can convey the meaning straight (if the name is good.)

BTW, AFAICS, the latest patch
doesn't have code to address this point.

Kirk-san, can you address this? I don't mind much if you add an argument or a new function.

Regards
Takayuki Tsunakawa

#81k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#80)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Monday, September 28, 2020 11:50 AM, Tsunakawa-san wrote:

From: Amit Kapila <amit.kapila16@gmail.com>

I agree with the above two points.

Thank you. I'm relieved to know I didn't misunderstand.

* Then, add a new function, say, smgrnblocks_cached() that simply
returns

the cached block count, and DropRelFileNodeBuffers() uses it instead
of smgrnblocks().

I am not sure if it worth adding a new function for this. Why not
simply add a boolean variable in smgrnblocks for this?

One reason is that adding an argument requires modification of existing call
sites (10 + a few). Another is that, although this may be different for each
person's taste, it's sometimes not easy to understand when a function call
with true/false appears. One such example is find_XXX(some_args,
true/false), where the true/false represents missing_ok. Another example is
as follows. I often wonder "what's the meaning of this false, and that true?"

if (!InstallXLogFileSegment(&destsegno, tmppath, false, 0, false))
elog(ERROR, "InstallXLogFileSegment should not have failed");

Fortunately, the new function is very short and doesn't duplicate much code.
The function is a simple getter and the function name can convey the
meaning straight (if the name is good.)

BTW, AFAICS, the latest patch
doesn't have code to address this point.

Kirk-san, can you address this? I don't mind much if you add an argument
or a new function.

I maybe missing something. so I'd like to check if my understanding is correct,
as I'm confused with what do we mean exactly by "cached value of nblocks".

Discussed upthread, smgrnblocks() does not always guarantee that it returns a
"cached" nblocks even in recovery.
When we enter this path in recovery path of DropRelFileNodeBuffers,
according to Tsunakawa-san:

* During recovery, DropRelFileNodeBuffers() gets the cached size of the relation fork. If it is cached, trust it and optimize the buffer invalidation. If it's not cached, we can't trust the return value of smgrnblocks() because it's the lseek(END) return value, so we avoid the optimization.

+ nTotalBlocks = smgrnblocks(smgr_reln, forkNum[j]);

But this comment in the smgrnblocks source code:
* For now, we only use cached values in recovery due to lack of a shared
* invalidation mechanism for changes in file size.
*/
if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
return reln->smgr_cached_nblocks[forknum];

So the nblocks returned in DropRelFileNodeBuffers are still not guaranteed to be "cached values"?
And that we want to add a new function (I think it's the lesser complicated way than modifying smgrnblocks):

/*
* smgrnblocksvalid() -- Calculate the number of blocks that are cached in
* the supplied relation.
*
* It is equivalent to calling smgrnblocks, but only used in recovery for now
* when DropRelFileNodeBuffers() is called, to ensure that only cached value
* is used, which is always valid.
*
* This returns an InvalidBlockNumber when smgr_cached_nblocks is not available
* and when isCached is false.
*/
BlockNumber
smgrnblocksvalid(SMgrRelation reln, ForkNumber forknum, bool isCached)
{
BlockNumber result;

/*
* For now, we only use cached values in recovery due to lack of a shared
* invalidation mechanism for changes in file size.
*/
if (InRecovery && if reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber
&& isCached)
return reln->smgr_cached_nblocks[forknum];
}

result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);

reln->smgr_cached_nblocks[forknum] = result;

if (!InRecovery && !isCached)
return InvalidBlockNumber;

return result;
}

Then in DropRelFileNodeBuffers
+ nTotalBlocks = smgrcachednblocks(smgr_reln, forkNum[j], true);

Is my understanding above correct?

Regards,
Kirk Jamison

#82tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#81)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Is my understanding above correct?

No. I simply meant DropRelFileNodeBuffers() calls the following function, and avoids the optimization if it returns InvalidBlockNumber.

BlockNumber
smgrcachednblocks(SMgrRelation reln, ForkNumber forknum)
{
return reln->smgr_cached_nblocks[forknum];
}

Regards
Takayuki Tsunakawa

#83k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#82)
2 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Monday, September 28, 2020 5:08 PM, Tsunakawa-san wrote:

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Is my understanding above correct?

No. I simply meant DropRelFileNodeBuffers() calls the following function,
and avoids the optimization if it returns InvalidBlockNumber.

BlockNumber
smgrcachednblocks(SMgrRelation reln, ForkNumber forknum) {
return reln->smgr_cached_nblocks[forknum];
}

Thank you for clarifying.

So in the new function, it goes something like:
if (InRecovery)
{
if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
return reln->smgr_cached_nblocks[forknum];
else
return InvalidBlockNumber;
}

I've revised the patch and added the new function accordingly in the attached file.
I also did not remove the duplicate code from smgrnblocks because Amit-san mentioned
that when the caching for non-recovery cases is implemented, we can use it
for non-recovery cases as well.

Although I am not sure if the way it's written in DropRelFileNodeBuffers is okay.
BlockNumberIsValid(nTotalBlocks)

nTotalBlocks = smgrcachednblocks(smgr_reln, forkNum[j]);
nBlocksToInvalidate = nTotalBlocks - firstDelBlock[j];

if (BlockNumberIsValid(nTotalBlocks) &&
nBlocksToInvalidate < BUF_DROP_FULLSCAN_THRESHOLD)
{
//enter optimization loop
}
else
{
//full scan for each fork
}

Regards,
Kirk Jamison

Attachments:

v17-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v17-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 005814c3521df425204fa26286c3a4bb0804fe77 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 11 Sep 2020 13:00:33 +0000
Subject: [PATCH] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork by calling smgrnblocks(), and it's safe
because there are no other processes but the startup process that
changes the relation size during recovery.  Otherwise, or if not in
recovery, proceed to sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 127 ++++++++++++++++++++++++++++++++++--
 src/backend/storage/smgr/smgr.c     |  36 +++++++++-
 src/include/storage/bufmgr.h        |   2 +-
 src/include/storage/smgr.h          |   1 +
 4 files changed, 157 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..8cb2209 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULLSCAN_THRESHOLD		(uint32)(NBuffers / 500)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,8 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+										 BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2965,18 +2969,27 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		XXX The relation might have extended before this, so this path is
+ *		only optimized during recovery when we can get a reliable cached
+ *		value of blocks for specified relation.  See comment in
+ *		smgrnblocks() in smgr.c.  In addition, it is safe to do this since
+ *		there are no other processes but the startup process that changes
+ *		the relation size during recovery.  Otherwise, or if not in
+ *		recovery, proceed to usual invalidation process, where it
+ *		sequentially searches the buffer pool.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2990,10 +3003,79 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	if (InRecovery)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber	nTotalBlocks; /* total nblocks */
+			BlockNumber nBlocksToInvalidate; /* total nblocks to be invalidated */
+
+			/* Get the total number of blocks for the supplied relation's fork */
+			nTotalBlocks = smgrcachednblocks(smgr_reln, forkNum[j]);
+
+			/* Get the total number of blocks to be invalidated for the specified fork */
+			nBlocksToInvalidate = nTotalBlocks - firstDelBlock[j];
+
+			/*
+			 * Do explicit hashtable probe if the blocks to be invalidated have not been
+			 * invalidated yet as well as if the ratio of total number of buffers to be
+			 * truncated against NBuffers is less than the threshold for full-scanning of
+			 * buffer pool. IOW, relation is small enough for its buffers to be removed.
+			 */
+			if (BlockNumberIsValid(nTotalBlocks) &&
+				nBlocksToInvalidate < BUF_DROP_FULLSCAN_THRESHOLD)
+			{
+				BlockNumber		curBlock;
+
+				for (curBlock = firstDelBlock[j]; curBlock < nTotalBlocks; curBlock++)
+				{
+					uint32		newHash;		/* hash value for newTag */
+					BufferTag	newTag;			/* identity of requested block */
+					LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+					int		buf_id;
+
+					/* create a tag so we can lookup the buffer */
+					INIT_BUFFERTAG(newTag, rnode.node, forkNum[j], curBlock);
+
+					/* determine its hash code and partition lock ID */
+					newHash = BufTableHashCode(&newTag);
+					newPartitionLock = BufMappingPartitionLock(newHash);
+
+					/* Check that it is in the buffer pool. If not, do nothing */
+					LWLockAcquire(newPartitionLock, LW_SHARED);
+					buf_id = BufTableLookup(&newTag, newHash);
+					LWLockRelease(newPartitionLock);
+
+					if (buf_id < 0)
+						continue;
+
+					bufHdr = GetBufferDescriptor(buf_id);
+
+					buf_state = LockBufHdr(bufHdr);
+
+					if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+						bufHdr->tag.forkNum == forkNum[j] &&
+						bufHdr->tag.blockNum == curBlock)
+						InvalidateBuffer(bufHdr);	/* releases spinlock */
+					else
+						UnlockBufHdr(bufHdr, buf_state);
+				}
+			}
+			else
+			{
+				/*
+				 * Relation is larger than the threshold. Execute full scan of
+				 * buffer pool for each fork.
+				 */
+				DropRelFileNodeBuffersOfFork(rnode.node, forkNum[j],
+											 firstDelBlock[j]);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3031,6 +3113,37 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 	}
 }
 
+
+/* -----------------------------------------------------------------
+ *		DropRelFileNodeBuffersOfFork
+ *
+ *		This function removes from the buffer pool the pages for
+ *		the specified relation's fork.
+ * -----------------------------------------------------------------
+ */
+static void
+DropRelFileNodeBuffersOfFork(RelFileNode rnode, ForkNumber forkNum,
+							 BlockNumber firstDelBlock)
+{
+	int			i;
+
+	for (i = 0; i < NBuffers; i++)
+	{
+		BufferDesc	*bufHdr = GetBufferDescriptor(i);
+		uint32		buf_state;
+
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+
 /* ---------------------------------------------------------------------
  *		DropRelFileNodesAllBuffers
  *
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..023f809 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -565,6 +565,40 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 }
 
 /*
+ *	smgrcachednblocks() -- Calculate the number of blocks that are cached in
+ *					 the supplied relation.
+ *
+ * It is equivalent to calling smgrnblocks, but only used in recovery for now
+ * when DropRelFileNodeBuffers() is called, to ensure that only cached value
+ * is used, which is always valid.
+ *
+ * This returns an InvalidBlockNumber when smgr_cached_nblocks is not available
+ */
+BlockNumber
+smgrcachednblocks(SMgrRelation reln, ForkNumber forknum)
+{
+	BlockNumber result;
+
+	/*
+	 * For now, we only use cached values in recovery due to lack of a shared
+	 * invalidation mechanism for changes in file size.
+	 */
+	if (InRecovery)
+	{
+		if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+			return reln->smgr_cached_nblocks[forknum];
+		else
+			return InvalidBlockNumber;
+	}
+
+	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
+
+	reln->smgr_cached_nblocks[forknum] = result;
+
+	return result;
+}
+
+/*
  *	smgrtruncate() -- Truncate the given forks of supplied relation to
  *					  each specified numbers of blocks
  *
@@ -583,7 +617,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..fbb7a4a 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -99,6 +99,7 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
 extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrcachednblocks(SMgrRelation reln, ForkNumber forknum);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchapplication/octet-stream; name=v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchDownload
From 5f3cef6674749b187054c5a9f2e7aae06001b536 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH] Prevent invalidating blocks in smgrextend() during recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

#84Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: k.jamison@fujitsu.com (#83)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Mon, 28 Sep 2020 08:57:36 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in

On Monday, September 28, 2020 5:08 PM, Tsunakawa-san wrote:

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Is my understanding above correct?

No. I simply meant DropRelFileNodeBuffers() calls the following function,
and avoids the optimization if it returns InvalidBlockNumber.

BlockNumber
smgrcachednblocks(SMgrRelation reln, ForkNumber forknum) {
return reln->smgr_cached_nblocks[forknum];
}

Thank you for clarifying.

FWIW, I (and maybe Amit) am thinking that the property we need here is
not it is cached or not but the accuracy of the returned file length,
and that the "cached" property should be hidden behind the API.

Another reason for not adding this function is the cached value is not
really reliable on non-recovery environment.

So in the new function, it goes something like:
if (InRecovery)
{
if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
return reln->smgr_cached_nblocks[forknum];
else
return InvalidBlockNumber;
}

If we add the new function, it should reutrn InvalidBlockNumber
without consulting smgr_nblocks().

I've revised the patch and added the new function accordingly in the attached file.
I also did not remove the duplicate code from smgrnblocks because Amit-san mentioned
that when the caching for non-recovery cases is implemented, we can use it
for non-recovery cases as well.

Although I am not sure if the way it's written in DropRelFileNodeBuffers is okay.
BlockNumberIsValid(nTotalBlocks)

nTotalBlocks = smgrcachednblocks(smgr_reln, forkNum[j]);
nBlocksToInvalidate = nTotalBlocks - firstDelBlock[j];

if (BlockNumberIsValid(nTotalBlocks) &&
nBlocksToInvalidate < BUF_DROP_FULLSCAN_THRESHOLD)
{
//enter optimization loop
}
else
{
//full scan for each fork
}

Hmm. The current loop in DropRelFileNodeBuffers looks like this:

if (InRecovery)
for (for each forks)
if (the fork meets the criteria)
<optimized dropping>
else
<full scan>

I think this is somewhat different from the current
discussion. Whether we sum-up the number of blcoks for all forks or
just use that of the main fork, we should take full scan if we failed
to know the accurate size for any one of the forks. (In other words,
it is stupid that we run a full scan for more than one fork at a
drop.)

Come to think of that, we can naturally sum-up all forks' blocks since
anyway we need to call smgrnblocks for all forks to know the
optimzation is usable.

So that block would be something like this:

for (forks of the rel)
/* the function returns InvalidBlockNumber if !InRecovery */
if (smgrnblocks returned InvalidBlockNumber)
total_blocks = InvalidBlockNumber;
break;
total_blocks += nbloks of this fork

/* <we could rely on the fact that InvalidBlockNumber is zero> */
if (total_blocks != InvalidBlockNumber && total_blocks < threshold)
for (forks of the rel)
for (blocks of the fork)
<try dropping the buffer for the block>
else
<full scan dropping>

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#85tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#83)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

I also did not remove the duplicate code from smgrnblocks because Amit-san
mentioned that when the caching for non-recovery cases is implemented, we
can use it for non-recovery cases as well.

But the extra code is not used now. The code for future usage should be added when it becomes necessary. Duplicate code may make people think that you should add an argument to smgrnblocks() instead of adding a new function.

+		if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+			return reln->smgr_cached_nblocks[forknum];
+		else
+			return InvalidBlockNumber;

Anyway, the else block is redundant, as the variable contains InvalidBlockNumber.

Also, as Amit-san mentioned, the cause of the slight performance regression when shared_buffers is small needs to be investigated and addressed. I think you can do it after sharing the performance result with a large shared_buffers.

I found no other problem.

Regards
Takayuki Tsunakawa

#86Amit Kapila
amit.kapila16@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#85)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Sep 29, 2020 at 7:21 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Also, as Amit-san mentioned, the cause of the slight performance regression when shared_buffers is small needs to be investigated and addressed.

Yes, I think it is mainly because extra instructions added in the
optimized code which doesn't make up for the loss when the size of
shared buffers is small.

--
With Regards,
Amit Kapila.

#87k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Kyotaro Horiguchi (#84)
2 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Tuesday, September 29, 2020 10:35 AM, Horiguchi-san wrote:

FWIW, I (and maybe Amit) am thinking that the property we need here is not it
is cached or not but the accuracy of the returned file length, and that the
"cached" property should be hidden behind the API.

Another reason for not adding this function is the cached value is not really
reliable on non-recovery environment.

So in the new function, it goes something like:
if (InRecovery)
{
if (reln->smgr_cached_nblocks[forknum] !=

InvalidBlockNumber)

return reln->smgr_cached_nblocks[forknum];
else
return InvalidBlockNumber;
}

If we add the new function, it should reutrn InvalidBlockNumber without
consulting smgr_nblocks().

So here's how I revised it
smgrcachednblocks(SMgrRelation reln, ForkNumber forknum)
{
if (InRecovery)
{
if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
return reln->smgr_cached_nblocks[forknum];
}
return InvalidBlockNumber;

Hmm. The current loop in DropRelFileNodeBuffers looks like this:

if (InRecovery)
for (for each forks)
if (the fork meets the criteria)
<optimized dropping>
else
<full scan>

I think this is somewhat different from the current discussion. Whether we
sum-up the number of blcoks for all forks or just use that of the main fork, we
should take full scan if we failed to know the accurate size for any one of the
forks. (In other words, it is stupid that we run a full scan for more than one
fork at a
drop.)

Come to think of that, we can naturally sum-up all forks' blocks since anyway
we need to call smgrnblocks for all forks to know the optimzation is usable.

I understand. We really don't have to enter the optimization when we know the
file size is inaccurate. That also makes the patch simpler.

So that block would be something like this:

for (forks of the rel)
/* the function returns InvalidBlockNumber if !InRecovery */
if (smgrnblocks returned InvalidBlockNumber)
total_blocks = InvalidBlockNumber;
break;
total_blocks += nbloks of this fork

/* <we could rely on the fact that InvalidBlockNumber is zero> */
if (total_blocks != InvalidBlockNumber && total_blocks < threshold)
for (forks of the rel)
for (blocks of the fork)
<try dropping the buffer for the block>
else
<full scan dropping>

I followed this logic in the attached patch.
Thank you very much for the thoughtful reviews.

Performance measurement for large shared buffers to follow.

Best regards,
Kirk Jamison

Attachments:

v18-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v18-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From c520ab04cbbe705c9a2d55c0575b2a6b84b7ced5 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 11 Sep 2020 13:00:33 +0000
Subject: [PATCH] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork by calling smgrcachednblocks(), and it's
safe because there are no other processes but the startup process
that changes the relation size during recovery.  Otherwise, or if
not in recovery, proceed to sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 145 +++++++++++++++++++++++++++---------
 src/backend/storage/smgr/smgr.c     |  25 ++++++-
 src/include/storage/bufmgr.h        |   2 +-
 src/include/storage/smgr.h          |   1 +
 4 files changed, 137 insertions(+), 36 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..d44686e 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULLSCAN_THRESHOLD		(uint32)(NBuffers / 500)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2965,18 +2967,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		XXX The relation might have extended before this, so this path is
+ *		only optimized during recovery when we can get a reliable cached
+ *		value of blocks for specified relation.  See comment in
+ *		smgrnblocks() in smgr.c.  In addition, it is safe to do this since
+ *		there are no other processes but the startup process that changes
+ *		the relation size during recovery.  Otherwise, or if not in
+ *		recovery, proceed to usual invalidation process, where it
+ *		sequentially searches the buffer pool.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	BlockNumber	nTotalBlocks;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2990,44 +3002,109 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
-	for (i = 0; i < NBuffers; i++)
+	/* get the total number of blocks for the supplied relation */
+	for (j = 0; j < nforks; j++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		BlockNumber	nForkBlocks; /* nblocks for a relation's fork */
 
-		/*
-		 * We can make this a tad faster by prechecking the buffer tag before
-		 * we attempt to lock the buffer; this saves a lot of lock
-		 * acquisitions in typical cases.  It should be safe because the
-		 * caller must have AccessExclusiveLock on the relation, or some other
-		 * reason to be certain that no one is loading new pages of the rel
-		 * into the buffer pool.  (Otherwise we might well miss such pages
-		 * entirely.)  Therefore, while the tag might be changing while we
-		 * look at it, it can't be changing *to* a value we care about, only
-		 * *away* from such a value.  So false negatives are impossible, and
-		 * false positives are safe because we'll recheck after getting the
-		 * buffer lock.
-		 *
-		 * We could check forkNum and blockNum as well as the rnode, but the
-		 * incremental win from doing so seems small.
-		 */
-		if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
-			continue;
+		 /* this returns an InvalidBlockNumber when not in recovery */
+		nForkBlocks = smgrcachednblocks(smgr_reln, forkNum[j]);
 
-		buf_state = LockBufHdr(bufHdr);
+		if (nForkBlocks == InvalidBlockNumber)
+		{
+			nTotalBlocks = InvalidBlockNumber;
+			break;
+		}
+		nTotalBlocks += nForkBlocks;
+	}
 
+	/*
+	 * Do explicit hashtable probe if the blocks to be dropped have not been
+	 * invalidated yet and the value is less than the full-scan threshold of
+	 * buffer pool.  IOW, relation is small enough for its buffers to be removed.
+	 */
+	if (nTotalBlocks != InvalidBlockNumber &&
+		nTotalBlocks < BUF_DROP_FULLSCAN_THRESHOLD)
+	{
 		for (j = 0; j < nforks; j++)
 		{
-			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-				bufHdr->tag.forkNum == forkNum[j] &&
-				bufHdr->tag.blockNum >= firstDelBlock[j])
+			BlockNumber		curBlock;
+
+			for (curBlock = firstDelBlock[j]; curBlock < nTotalBlocks; curBlock++)
 			{
-				InvalidateBuffer(bufHdr);	/* releases spinlock */
-				break;
+				uint32		newHash;		/* hash value for newTag */
+				BufferTag	newTag;			/* identity of requested block */
+				LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(newTag, rnode.node, forkNum[j], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				newHash = BufTableHashCode(&newTag);
+				newPartitionLock = BufMappingPartitionLock(newHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing. */
+				LWLockAcquire(newPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&newTag, newHash);
+				LWLockRelease(newPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum == curBlock)
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
 			}
 		}
-		if (j >= nforks)
-			UnlockBufHdr(bufHdr, buf_state);
+	}
+	else
+	{
+		for (i= 0; i < NBuffers; i++)
+		{
+			bufHdr = GetBufferDescriptor(i);
+
+			/*
+			 * We can make this a tad faster by prechecking the buffer tag before
+			 * we attempt to lock the buffer; this saves a lot of lock
+			 * acquisitions in typical cases.  It should be safe because the
+			 * caller must have AccessExclusiveLock on the relation, or some other
+			 * reason to be certain that no one is loading new pages of the rel
+			 * into the buffer pool.  (Otherwise we might well miss such pages
+			 * entirely.)  Therefore, while the tag might be changing while we
+			 * look at it, it can't be changing *to* a value we care about, only
+			 * *away* from such a value.  So false negatives are impossible, and
+			 * false positives are safe because we'll recheck after getting the
+			 * buffer lock.
+			 *
+			 * We could check forkNum and blockNum as well as the rnode, but the
+			 * incremental win from doing so seems small.
+			 */
+			if (!RelFileNodeEquals(bufHdr->tag.rnode, rnode.node))
+				continue;
+
+			buf_state = LockBufHdr(bufHdr);
+
+			for (j = 0; j < nforks; j++)
+			{
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= firstDelBlock[j])
+				{
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+					break;
+				}
+			}
+			if (j >= nforks)
+				UnlockBufHdr(bufHdr, buf_state);
+		}
 	}
 }
 
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..4e45c47 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -565,6 +565,29 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 }
 
 /*
+ *	smgrcachednblocks() -- Calculate the number of blocks that are cached in
+ *					 the supplied relation.
+ *
+ * It is equivalent to calling smgrnblocks, but only used in recovery for now
+ * when DropRelFileNodeBuffers() is called.  This ensures that only cached value
+ * is used which is always valid in recovery, since there is no shared
+ * invalidation mechanism that is implemented yet for changes in file size.
+ *
+ * This returns an InvalidBlockNumber when smgr_cached_nblocks is not available
+ * and when the path is not in recovery.
+ */
+BlockNumber
+smgrcachednblocks(SMgrRelation reln, ForkNumber forknum)
+{
+	if (InRecovery)
+	{
+		if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+			return reln->smgr_cached_nblocks[forknum];
+	}
+	return InvalidBlockNumber;
+}
+
+/*
  *	smgrtruncate() -- Truncate the given forks of supplied relation to
  *					  each specified numbers of blocks
  *
@@ -583,7 +606,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..fbb7a4a 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -99,6 +99,7 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
 extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrcachednblocks(SMgrRelation reln, ForkNumber forknum);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchapplication/octet-stream; name=v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchDownload
From 9266f25c02818feab55d2a5b524fba5f0f305b28 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH] Prevent invalidating blocks in smgrextend() during recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

#88k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#87)
3 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi,

I revised the patch again. Attached is V19.
The previous patch's algorithm missed entering the optimization loop.
So I corrected that and removed the extra function I added
in the previous versions.

The revised patch goes something like this:
for (forks of rel)
{
if (smgrcachednblocks() == InvalidBlockNumber)
break; //go to full scan
if (nBlocksToInvalidate < buf_full_scan_threshold)
for (blocks of the fork)
else
break; //go to full scan
}
<execute full scan>

Recovery performance measurement results below.
But it seems there are overhead even with large shared buffers.

| s_b | master | patched | %reg |
|-------|--------|---------|-------|
| 128MB | 36.052 | 39.451 | 8.62% |
| 1GB | 21.731 | 21.73 | 0.00% |
| 20GB | 24.534 | 25.137 | 2.40% |
| 100GB | 30.54 | 31.541 | 3.17% |

I'll investigate further. Or if you have any feedback or advice, I'd appreciate it.

Machine specs used for testing:
RHEL7, 8 core, 256 GB RAM, xfs

Configuration:
wal_level = replica
autovacuum = off
full_page_writes = off

# For streaming replication from primary.
synchronous_commit = remote_write
synchronous_standby_names = ''

# For Standby.
#hot_standby = on
#primary_conninfo

shared_buffers = 128MB
# 1GB, 20GB, 100GB

Just in case it helps for some understanding,
I also attached the recovery log 018_wal_optimize_node_replica.log
with some ereport that prints whether we enter the optimization loop or do full scan.

Regards,
Kirk Jamison

Attachments:

v19-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v19-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From b8b4b1e865a74a846c5375f257c0c1818db726fe Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 11 Sep 2020 13:00:33 +0000
Subject: [PATCH] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 86 ++++++++++++++++++++++++++++++++++---
 src/backend/storage/smgr/smgr.c     | 23 +++++++++-
 src/include/storage/bufmgr.h        |  2 +-
 src/include/storage/smgr.h          |  1 +
 4 files changed, 103 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..22f51c4 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULLSCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2965,18 +2967,26 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		XXX The relation might have extended before this, so this path is
+ *		only optimized during recovery when we can get a reliable cached
+ *		value of blocks for specified relation.  In addition, it is safe to
+ *		do this since there are no other processes but the startup process
+ *		that changes the relation size during recovery.  Otherwise, or if
+ *		not in recovery, proceed to usual invalidation process, where it
+ *		sequentially searches the buffer pool.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2990,10 +3000,72 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/* Enter optimization loop if blocks are valid, wherein the path is in recovery */
+	for (j = 0; j < nforks; j++)
+	{
+		BlockNumber	nTotalBlocks; /* total nblocks */
+		BlockNumber nBlocksToInvalidate; /* total nblocks to be invalidated */
+
+		/* Get the total number of blocks for the supplied relation's fork */
+		nTotalBlocks = smgrcachednblocks(smgr_reln, forkNum[j]);
+
+		/* If blocks are invalid, exit the optimization and execute full scan */
+		if (nTotalBlocks == InvalidBlockNumber)
+			break;
+
+		/* Get the total number of blocks to be invalidated for the specified fork */
+		nBlocksToInvalidate = nTotalBlocks - firstDelBlock[j];
+
+		/*
+		 * Do explicit hashtable probe if the blocks to be invalidated have not been
+		 * invalidated yet as well as if the ratio of total number of buffers to be
+		 * truncated against NBuffers is less than the threshold for full-scanning of
+		 * buffer pool. IOW, relation is small enough for its buffers to be removed.
+		 */
+		if (nBlocksToInvalidate < BUF_DROP_FULLSCAN_THRESHOLD)
+		{
+			BlockNumber		curBlock;
+
+			for (curBlock = firstDelBlock[j]; curBlock < nTotalBlocks; curBlock++)
+			{
+				uint32		newHash;		/* hash value for newTag */
+				BufferTag	newTag;			/* identity of requested block */
+				LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(newTag, rnode.node, forkNum[j], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				newHash = BufTableHashCode(&newTag);
+				newPartitionLock = BufMappingPartitionLock(newHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing */
+				LWLockAcquire(newPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&newTag, newHash);
+				LWLockRelease(newPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum == curBlock)
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
+			}
+		}
+		else
+			break;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..e7a9ded 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -565,6 +565,27 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 }
 
 /*
+ *	smgrcachednblocks() -- Calculate the number of blocks that are cached in
+ *					 the supplied relation.
+ *
+ * It is equivalent to calling smgrnblocks, but only used in recovery for now
+ * when DropRelFileNodeBuffers() is called.  This ensures that only cached value
+ * is used which is always valid in recovery, since there is no shared
+ * invalidation mechanism that is implemented yet for changes in file size.
+ *
+ * This returns an InvalidBlockNumber when smgr_cached_nblocks is not available
+ * and when not in recovery.
+ */
+BlockNumber
+smgrcachednblocks(SMgrRelation reln, ForkNumber forknum)
+{
+	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+			return reln->smgr_cached_nblocks[forknum];
+
+	return InvalidBlockNumber;
+}
+
+/*
  *	smgrtruncate() -- Truncate the given forks of supplied relation to
  *					  each specified numbers of blocks
  *
@@ -583,7 +604,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..fbb7a4a 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -99,6 +99,7 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
 extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrcachednblocks(SMgrRelation reln, ForkNumber forknum);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchapplication/octet-stream; name=v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchDownload
From 9266f25c02818feab55d2a5b524fba5f0f305b28 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH] Prevent invalidating blocks in smgrextend() during recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

018_wal_optimize_node_replica.logapplication/octet-stream; name=018_wal_optimize_node_replica.logDownload
#89tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#88)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Recovery performance measurement results below.
But it seems there are overhead even with large shared buffers.

| s_b | master | patched | %reg |
|-------|--------|---------|-------|
| 128MB | 36.052 | 39.451 | 8.62% |
| 1GB | 21.731 | 21.73 | 0.00% |
| 20GB | 24.534 | 25.137 | 2.40% |
| 100GB | 30.54 | 31.541 | 3.17% |

Did you really check that the optimization path is entered and the traditional path is never entered?

With the following code, when the main fork does not meet the optimization criteria, other forks are not optimized as well. You want to determine each fork's optimization separately, don't you?

+		/* If blocks are invalid, exit the optimization and execute full scan */
+		if (nTotalBlocks == InvalidBlockNumber)
+			break;
+		else
+			break;
+	}
 	for (i = 0; i < NBuffers; i++)

Regards
Takayuki Tsunakawa

#90Amit Kapila
amit.kapila16@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#89)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Oct 1, 2020 at 8:11 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Recovery performance measurement results below.
But it seems there are overhead even with large shared buffers.

| s_b | master | patched | %reg |
|-------|--------|---------|-------|
| 128MB | 36.052 | 39.451 | 8.62% |
| 1GB | 21.731 | 21.73 | 0.00% |
| 20GB | 24.534 | 25.137 | 2.40% |
| 100GB | 30.54 | 31.541 | 3.17% |

Did you really check that the optimization path is entered and the traditional path is never entered?

I have one idea for performance testing. We can even test this for
non-recovery paths by removing the recovery-related check like only
use it when there are cached blocks. You can do this if testing via
recovery path is difficult because at the end performance should be
same for recovery and non-recovery paths.

--
With Regards,
Amit Kapila.

#91tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Amit Kapila (#90)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Amit Kapila <amit.kapila16@gmail.com>

I have one idea for performance testing. We can even test this for
non-recovery paths by removing the recovery-related check like only
use it when there are cached blocks. You can do this if testing via
recovery path is difficult because at the end performance should be
same for recovery and non-recovery paths.

That's a good idea.

Regards
Takayuki Tsunakawa

#92Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#89)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 1 Oct 2020 02:40:52 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

With the following code, when the main fork does not meet the
optimization criteria, other forks are not optimized as well. You
want to determine each fork's optimization separately, don't you?

In more detail, if smgrcachednblocks() returned InvalidBlockNumber for
any of the forks, we should give up the optimization at all since we
need to run a full scan anyway. On the other hand, if any of the
forks is smaller than the threshold, we still can use the optimization
when we know the accurate block number of all the forks.

Still, I prefer to use total block number of all forks since we anyway
visit the all forks. Is there any reason to exlucde forks other than
the main fork while we visit all of them already?

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#93tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Kyotaro Horiguchi (#92)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

In more detail, if smgrcachednblocks() returned InvalidBlockNumber for
any of the forks, we should give up the optimization at all since we
need to run a full scan anyway. On the other hand, if any of the
forks is smaller than the threshold, we still can use the optimization
when we know the accurate block number of all the forks.

Ah, I got your point (many eyes in open source development is nice.) Still, I feel it's better to treat each fork separately, because the inner loop in the traditional path may be able to skip forks that have been already processed in the optimization path. For example, if the forks[] array contains {fsm, vm, main} in this order (I know main is usually put at the beginning), fsm and vm are processed in the optimization path and the inner loop in the traditional path can skip fsm and vm.

Still, I prefer to use total block number of all forks since we anyway
visit the all forks. Is there any reason to exlucde forks other than
the main fork while we visit all of them already?

When the number of cached blocks for a main fork is below the threshold but the total cached blocks of all forks exceeds the threshold, the optimization is skipped. I think it's mottainai.

Regards
Takayuki Tsunakawa

#94Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#93)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 1 Oct 2020 04:20:27 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

In more detail, if smgrcachednblocks() returned InvalidBlockNumber for
any of the forks, we should give up the optimization at all since we
need to run a full scan anyway. On the other hand, if any of the
forks is smaller than the threshold, we still can use the optimization
when we know the accurate block number of all the forks.

Ah, I got your point (many eyes in open source development is nice.) Still, I feel it's better to treat each fork separately, because the inner loop in the traditional path may be able to skip forks that have been already processed in the optimization path. For example, if the forks[] array contains {fsm, vm, main} in this order (I know main is usually put at the beginning), fsm and vm are processed in the optimization path and the inner loop in the traditional path can skip fsm and vm.

I thought that the advantage of this optimization is that we don't
need to visit all buffers? If we need to run a full-scan for any
reason, there's no point in looking-up already-visited buffers
again. That's just wastefull cycles. Am I missing somethig?

Still, I prefer to use total block number of all forks since we anyway
visit the all forks. Is there any reason to exlucde forks other than
the main fork while we visit all of them already?

When the number of cached blocks for a main fork is below the threshold but the total cached blocks of all forks exceeds the threshold, the optimization is skipped. I think it's mottainai.

I don't understand. If we chose to the optimized dropping, the reason
is the number of buffer lookup is fewer than a certain threashold. Why
do you think that the fork kind a buffer belongs to is relevant to the
criteria?

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#95k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#90)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, October 1, 2020 11:49 AM, Amit Kapila wrote:

On Thu, Oct 1, 2020 at 8:11 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Recovery performance measurement results below.
But it seems there are overhead even with large shared buffers.

| s_b | master | patched | %reg |
|-------|--------|---------|-------|
| 128MB | 36.052 | 39.451 | 8.62% |
| 1GB | 21.731 | 21.73 | 0.00% |
| 20GB | 24.534 | 25.137 | 2.40% | 100GB | 30.54 | 31.541 |
| 3.17% |

Did you really check that the optimization path is entered and the traditional

path is never entered?

Oops. Thanks Tsunakawa-san for catching that.
Will fix in the next patch, replacing break with continue.

I have one idea for performance testing. We can even test this for
non-recovery paths by removing the recovery-related check like only use it
when there are cached blocks. You can do this if testing via recovery path is
difficult because at the end performance should be same for recovery and
non-recovery paths.

For non-recovery path, did you mean by any chance
measuring the cache hit rate for varying shared_buffers?

SELECT
sum(heap_blks_read) as heap_read,
sum(heap_blks_hit) as heap_hit,
sum(heap_blks_hit) / (sum(heap_blks_hit) + sum(heap_blks_read)) as ratio
FROM
pg_statio_user_tables;

Regards,
Kirk Jamison

#96tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#95)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

For non-recovery path, did you mean by any chance
measuring the cache hit rate for varying shared_buffers?

No. You can test the speed of DropRelFileNodeBuffers() during normal operation, i.e. by running TRUNCATE on psql, instead of performing recovery. To enable that, you can just remove the checks for recovery, i.e. removing the check if InRecovery and if the value is cached or not.

Regards
Takayuki Tsunakawa

#97tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Kyotaro Horiguchi (#94)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

I thought that the advantage of this optimization is that we don't
need to visit all buffers? If we need to run a full-scan for any
reason, there's no point in looking-up already-visited buffers
again. That's just wastefull cycles. Am I missing somethig?

I don't understand. If we chose to the optimized dropping, the reason
is the number of buffer lookup is fewer than a certain threashold. Why
do you think that the fork kind a buffer belongs to is relevant to the
criteria?

I rethought about this, and you certainly have a point, but... OK, I think I understood. I should have thought in a complicated way. In other words, you're suggesting "Let's simply treat all forks as one relation to determine whether to optimize," right? That is, the code simple becomes:

Sums up the number of buffers to invalidate in all forks;
if (the cached sizes of all forks are valid && # of buffers to invalidate < THRESHOLD)
{
do the optimized way;
return;
}
do the traditional way;

This will be simple, and I'm +1.

Regards
Takayuki Tsunakawa

#98k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#97)
2 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, October 1, 2020 4:52 PM, Tsunakawa-san wrote:

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

I thought that the advantage of this optimization is that we don't
need to visit all buffers? If we need to run a full-scan for any
reason, there's no point in looking-up already-visited buffers again.
That's just wastefull cycles. Am I missing somethig?

I don't understand. If we chose to the optimized dropping, the reason
is the number of buffer lookup is fewer than a certain threashold. Why
do you think that the fork kind a buffer belongs to is relevant to the
criteria?

I rethought about this, and you certainly have a point, but... OK, I think I
understood. I should have thought in a complicated way. In other words,
you're suggesting "Let's simply treat all forks as one relation to determine
whether to optimize," right? That is, the code simple becomes:

Sums up the number of buffers to invalidate in all forks; if (the cached sizes
of all forks are valid && # of buffers to invalidate < THRESHOLD) {
do the optimized way;
return;
}
do the traditional way;

This will be simple, and I'm +1.

This is actually close to the v18 I posted trying Horiguchi-san's approach, but that
patch had bug. So attached is an updated version (v20) trying this approach again.
I hope it's bug-free this time.

Regards,
Kirk Jamison

Attachments:

v20-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v20-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 1f4a85a0c57fcd00af530f0932991775b9f532f7 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 11 Sep 2020 13:00:33 +0000
Subject: [PATCH] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 92 ++++++++++++++++++++++++++++++++++---
 src/backend/storage/smgr/smgr.c     | 23 +++++++++-
 src/include/storage/bufmgr.h        |  2 +-
 src/include/storage/smgr.h          |  1 +
 4 files changed, 109 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..1b72510 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULLSCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2965,18 +2967,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		XXX The relation might have extended before this, so this path is
+ *		only optimized during recovery when we can get a reliable cached
+ *		value of blocks for specified relation.  In addition, it is safe to
+ *		do this since there are no other processes but the startup process
+ *		that changes the relation size during recovery.  Otherwise, or if
+ *		not in recovery, proceed to usual invalidation process, where it
+ *		sequentially searches the buffer pool.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	BlockNumber	nBlocksToInvalidate = 0;
+	BlockNumber	nTotalBlocks = 0;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2990,10 +3002,76 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of cached blocks and to-be-invalidated blocks
+	 * of the relation.  If a fork's nblocks is not valid, break the loop.
+	 */
+	for (j = 0; j < nforks; j++)
+	{
+		BlockNumber	nForkBlocks; /* total nblocks for a relation's fork */
+
+		nForkBlocks = smgrcachednblocks(smgr_reln, forkNum[j]);
+
+		if (nForkBlocks == InvalidBlockNumber)
+		{
+			nTotalBlocks = InvalidBlockNumber;
+			break;
+		}
+		nTotalBlocks += nForkBlocks;
+		nBlocksToInvalidate = nTotalBlocks - firstDelBlock[j];
+	}
+
+	/*
+	 * Do explicit hashtable probe if the total of nblocks of relation's forks
+	 * is not invalid and the nblocks to be invalidated is less than the full-scan
+	 * threshold of buffer pool.  Otherwise, full scan is executed.
+	 */
+	if (nTotalBlocks != InvalidBlockNumber &&
+		nBlocksToInvalidate < BUF_DROP_FULLSCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		curBlock;
+
+			for (curBlock = firstDelBlock[j]; curBlock < nTotalBlocks; curBlock++)
+			{
+				uint32		newHash;		/* hash value for newTag */
+				BufferTag	newTag;			/* identity of requested block */
+				LWLock	   	*newPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(newTag, rnode.node, forkNum[j], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				newHash = BufTableHashCode(&newTag);
+				newPartitionLock = BufMappingPartitionLock(newHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing. */
+				LWLockAcquire(newPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&newTag, newHash);
+				LWLockRelease(newPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum == curBlock)
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..69f431c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -565,6 +565,27 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 }
 
 /*
+ *	smgrcachednblocks() -- Calculate the number of blocks that are cached in
+ *					 the supplied relation.
+ *
+ * It is equivalent to calling smgrnblocks, but only used in recovery for now
+ * when DropRelFileNodeBuffers() is called.  This ensures that only cached value
+ * is used which is always valid in recovery, since there is no shared
+ * invalidation mechanism that is implemented yet for changes in file size.
+ *
+ * This returns an InvalidBlockNumber when smgr_cached_nblocks is not available
+ * and when not in recovery.
+ */
+BlockNumber
+smgrcachednblocks(SMgrRelation reln, ForkNumber forknum)
+{
+	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+		return reln->smgr_cached_nblocks[forknum];
+
+	return InvalidBlockNumber;
+}
+
+/*
  *	smgrtruncate() -- Truncate the given forks of supplied relation to
  *					  each specified numbers of blocks
  *
@@ -583,7 +604,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..fbb7a4a 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -99,6 +99,7 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
 extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrcachednblocks(SMgrRelation reln, ForkNumber forknum);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchapplication/octet-stream; name=v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchDownload
From 6b0219580f34a3c0b63445521d6ba1fef3352f53 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH] Prevent invalidating blocks in smgrextend() during recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

#99Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: k.jamison@fujitsu.com (#98)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 1 Oct 2020 12:55:34 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in

On Thursday, October 1, 2020 4:52 PM, Tsunakawa-san wrote:

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

I thought that the advantage of this optimization is that we don't
need to visit all buffers? If we need to run a full-scan for any
reason, there's no point in looking-up already-visited buffers again.
That's just wastefull cycles. Am I missing somethig?

I don't understand. If we chose to the optimized dropping, the reason
is the number of buffer lookup is fewer than a certain threashold. Why
do you think that the fork kind a buffer belongs to is relevant to the
criteria?

I rethought about this, and you certainly have a point, but... OK, I think I
understood. I should have thought in a complicated way. In other words,
you're suggesting "Let's simply treat all forks as one relation to determine
whether to optimize," right? That is, the code simple becomes:

Exactly. The concept of the threshold is that if we are expected to
repeat buffer look-up than that, we consider just one-time full-scan
more efficient. Since we know we are going to drop buffers of all (or
the specified) forks of the relation at once, the number of looking-up
is naturally the sum of the expected number of the buffers of all
forks.

whether to optimize," right? That is, the code simple becomes:

Sums up the number of buffers to invalidate in all forks;
if (the cached sizes
of all forks are valid && # of buffers to invalidate < THRESHOLD) {
do the optimized way;
return;
}
do the traditional way;

This will be simple, and I'm +1.

Thanks!

This is actually close to the v18 I posted trying Horiguchi-san's approach, but that
patch had bug. So attached is an updated version (v20) trying this approach again.
I hope it's bug-free this time.

Thaks for the new version.

- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		XXX The relation might have extended before this, so this path is

The following description is found in the comment for FlushRelationBuffers.

* XXX currently it sequentially searches the buffer pool, should be
* changed to more clever ways of searching. This routine is not
* used in any performance-critical code paths, so it's not worth
* adding additional overhead to normal paths to make it go faster;
* but see also DropRelFileNodeBuffers.

This looks like to me "We won't do that kind of optimization for
FlushRelationBuffers, but DropRelFileNodeBuffers would need it". If
so, don't we need to revise the comment together?

- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		XXX The relation might have extended before this, so this path is
+ *		only optimized during recovery when we can get a reliable cached
+ *		value of blocks for specified relation.  In addition, it is safe to
+ *		do this since there are no other processes but the startup process
+ *		that changes the relation size during recovery.  Otherwise, or if
+ *		not in recovery, proceed to usual invalidation process, where it
+ *		sequentially searches the buffer pool.

This should no longer be a XXX comment. It seems to me somewhat
describing too-detailed at this function's level. How about something
like the follwoing? (excpet its syntax, or phrasing:p)

===
If the expected maximum number of buffers to drop is small enough
compared to NBuffers, individual buffers are located by
BufTableLookup. Otherwise we scan through all buffers. Snnce we
mustn't leave a buffer behind, we take the latter way unless the
number is not reliably identified. See smgrcachednblocks() for
details.
===

(I'm still mildly opposed to the function name, which seems exposing
detail too much.)

+	 * Get the total number of cached blocks and to-be-invalidated blocks
+	 * of the relation.  If a fork's nblocks is not valid, break the loop.

The number of file blocks is not usually equal to the number of
existing buffers for the file. We might need to explain that
limitation here.

+ for (j = 0; j < nforks; j++)

Though I understand that j is considered to be in a connection with
fork number, I'm a bit uncomfortable that j is used for the outmost
loop..

+ for (curBlock = firstDelBlock[j]; curBlock < nTotalBlocks; curBlock++)

Mmm. We should compare curBlock with the number of blocks of the fork,
not the total of all forks.

+				uint32		newHash;		/* hash value for newTag */
+				BufferTag	newTag;			/* identity of requested block */
+				LWLock	   	*newPartitionLock;	/* buffer partition lock for it */

It seems to be copied from somewhere, but the buffer is not new at
all.

+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum == curBlock)
+					InvalidateBuffer(bufHdr);	/* releases spinlock */

I think it cannot happen that the block is used for a different block
of the same relation-fork, but it could be safer to check
bufHdr->tag.blockNum >= firstDelBlock[j] instead.

+/*
+ *	smgrcachednblocks() -- Calculate the number of blocks that are cached in
+ *					 the supplied relation.
+ *
+ * It is equivalent to calling smgrnblocks, but only used in recovery for now
+ * when DropRelFileNodeBuffers() is called.  This ensures that only cached value
+ * is used which is always valid in recovery, since there is no shared
+ * invalidation mechanism that is implemented yet for changes in file size.
+ *
+ * This returns an InvalidBlockNumber when smgr_cached_nblocks is not available
+ * and when not in recovery.

Isn't it too concrete? We need to mention the buggy-kernel issue here
rahter than that of callers.

And if the comment is correct, we should Assert(InRecovery) at the
beggining of this function.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#100Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Kyotaro Horiguchi (#99)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Fri, 02 Oct 2020 11:44:46 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

At Thu, 1 Oct 2020 12:55:34 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in 
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		XXX The relation might have extended before this, so this path is
+ *		only optimized during recovery when we can get a reliable cached
+ *		value of blocks for specified relation.  In addition, it is safe to
+ *		do this since there are no other processes but the startup process
+ *		that changes the relation size during recovery.  Otherwise, or if
+ *		not in recovery, proceed to usual invalidation process, where it
+ *		sequentially searches the buffer pool.

This should no longer be a XXX comment. It seems to me somewhat
describing too-detailed at this function's level. How about something
like the follwoing? (excpet its syntax, or phrasing:p)

===
If the expected maximum number of buffers to drop is small enough
compared to NBuffers, individual buffers are located by
BufTableLookup. Otherwise we scan through all buffers. Snnce we
mustn't leave a buffer behind, we take the latter way unless the
number is not reliably identified. See smgrcachednblocks() for
details.
===

The second to last phrase is inversed, and some typos are found. FWIW
this is the revised version.

====
If we are expected to drop buffers less enough, we locate individual
buffers using BufTableLookup. Otherwise we scan through all
buffers. Since we mustn't leave a buffer behind, we take the latter
way unless the sizes of all the involved forks are known to be
accurte. See smgrcachednblocks() for details.
====

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#101k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Kyotaro Horiguchi (#99)
2 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Friday, October 2, 2020 11:45 AM, Horiguchi-san wrote:

Thaks for the new version.

Thank you for your thoughtful reviews!
I've attached an updated patch addressing the comments below.

1.

The following description is found in the comment for FlushRelationBuffers.

* XXX currently it sequentially searches the buffer pool, should

be

* changed to more clever ways of searching. This routine is

not

* used in any performance-critical code paths, so it's not worth
* adding additional overhead to normal paths to make it go

faster;

* but see also DropRelFileNodeBuffers.

This looks like to me "We won't do that kind of optimization for
FlushRelationBuffers, but DropRelFileNodeBuffers would need it". If so,
don't we need to revise the comment together?

Yes, but instead of combining, I just removed the comment in FlushRelationBuffers that mentions
referring to DropRelFileNodeBuffers. I think it meant the same of using more clever ways of searching.
But that comment s not applicable anymore in DropRelFileNodeBuffers due to the optimization.
- *             adding additional overhead to normal paths to make it go faster;
- *             but see also DropRelFileNodeBuffers.
+ *             adding additional overhead to normal paths to make it go faster.

2.

- * XXX currently it sequentially searches the buffer pool, should be
- * changed to more clever ways of searching. However, this routine
- * is used only in code paths that aren't very performance-critical,
- * and we shouldn't slow down the hot paths to make it faster ...

I revised and removed most parts of this code comment in the DropRelFileNodeBuffers
because isn't it the point of the optimization, to make the path faster for some performance
cases we've tackled in the thread?

3.

This should no longer be a XXX comment.

Alright. I've fixed it.

4.

It seems to me somewhat
describing too-detailed at this function's level. How about something like the
follwoing? (excpet its syntax, or phrasing:p)
====
If we are expected to drop buffers less enough, we locate individual buffers
using BufTableLookup. Otherwise we scan through all buffers. Since we
mustn't leave a buffer behind, we take the latter way unless the sizes of all the
involved forks are known to be accurte. See smgrcachednblocks() for details.
====

Sure. I paraphrased it like below.

If the expected maximum number of buffers to be dropped is small
enough, individual buffer is located by BufTableLookup(). Otherwise,
the buffer pool is sequentially scanned. Since buffers must not be
left behind, the latter way is executed unless the sizes of all the
involved forks are known to be accurate. See smgrcachednblocks() for
more details.

5.

(I'm still mildly opposed to the function name, which seems exposing detail
too much.)

I can't think of a better name, but smgrcachednblocks seems straightforward though.
Although I understand that it may be confused with the relation property smgr_cached_nblocks.
But isn't that what we're getting in the function?

6.

+	 * Get the total number of cached blocks and to-be-invalidated
blocks
+	 * of the relation.  If a fork's nblocks is not valid, break the loop.

The number of file blocks is not usually equal to the number of existing
buffers for the file. We might need to explain that limitation here.

I revised that comment like below..

Get the total number of cached blocks and to-be-invalidated blocks
of the relation. The cached value returned by smgrcachednblocks
could be smaller than the actual number of existing buffers of the
file. This is caused by buggy Linux kernels that might not have
accounted the recent write. If a fork's nblocks is invalid, exit loop.

7.

+ for (j = 0; j < nforks; j++)

Though I understand that j is considered to be in a connection with fork
number, I'm a bit uncomfortable that j is used for the outmost loop..

I agree. We must use I for the outer loop for consistency.

8.

+			for (curBlock = firstDelBlock[j]; curBlock <
nTotalBlocks;
+curBlock++)

Mmm. We should compare curBlock with the number of blocks of the fork,
not the total of all forks.

Oops. Yes. That should be nForkBlocks, so we have to call again smgrcachednblocks()
In the optimization loop for forks.

9.

+				uint32		newHash;		/*hash value for newTag */
+				BufferTag	newTag; 	/* identity of requested block */
+				LWLock	   	*newPartitionLock; 	/* buffer partition lock for it */

It seems to be copied from somewhere, but the buffer is not new at all.

Thanks for catching that. Yeah. Fixed.

10.

+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum == curBlock)
+					InvalidateBuffer(bufHdr);	/* releases spinlock */

I think it cannot happen that the block is used for a different block of the
same relation-fork, but it could be safer to check
bufHdr->tag.blockNum >= firstDelBlock[j] instead.

Understood and that's fine with me. Updated.

11.

+ *	smgrcachednblocks() -- Calculate the number of blocks that are
cached in
+ *					 the supplied relation.
+ *
+ * It is equivalent to calling smgrnblocks, but only used in recovery
+for now
+ * when DropRelFileNodeBuffers() is called.  This ensures that only
+cached value
+ * is used which is always valid in recovery, since there is no shared
+ * invalidation mechanism that is implemented yet for changes in file size.
+ *
+ * This returns an InvalidBlockNumber when smgr_cached_nblocks is not
+available
+ * and when not in recovery.

Isn't it too concrete? We need to mention the buggy-kernel issue here rahter
than that of callers.

And if the comment is correct, we should Assert(InRecovery) at the beggining
of this function.

I did not add the assert because it causes the recovery tap test to fail.
However, I updated the function description like below.

It is equivalent to calling smgrnblocks, but only used in recovery for now.
The returned value of file size could be inaccurate because the lseek of buggy
Linux kernels might not have accounted the recent file extension or write.
However, this function ensures that cached values are only used in recovery,
since there is no shared invalidation mechanism that is implemented yet for
changes in file size.

This returns an InvalidBlockNumber when smgr_cached_nblocks is not available
and when not in recovery.

Thanks a lot for the reviews.
If there are any more comments, feedback, or points I might have missed please feel free to reply.

Best regards,
Kirk Jamison

Attachments:

v21-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v21-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 50788405a308a910cee08ac4bef414e002ba4a23 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 11 Sep 2020 13:00:33 +0000
Subject: [PATCH] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 99 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     | 25 +++++++++-
 src/include/storage/bufmgr.h        |  2 +-
 src/include/storage/smgr.h          |  1 +
 4 files changed, 116 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..e2d6219 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2965,18 +2967,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are known to be accurate. See smgrcachednblocks() for
+ *		more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	BlockNumber	nBlocksToInvalidate = 0;
+	BlockNumber	nForkBlocks = 0;
+	BlockNumber	nTotalBlocks = 0;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2990,10 +3002,80 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of cached blocks and to-be-invalidated blocks
+	 * of the relation.  The cached value returned by smgrcachednblocks
+	 * could be smaller than the actual number of existing buffers of the
+	 * file.  This is caused by buggy Linux kernels that might not have
+	 * accounted the recent write.  If a fork's nblocks is invalid, exit loop.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the total nblocks for a relation's fork */
+		nForkBlocks = smgrcachednblocks(smgr_reln, forkNum[i]);
+
+		if (nForkBlocks == InvalidBlockNumber)
+		{
+			nTotalBlocks = InvalidBlockNumber;
+			break;
+		}
+		nTotalBlocks += nForkBlocks;
+		nBlocksToInvalidate = nTotalBlocks - firstDelBlock[i];
+	}
+
+	/*
+	 * Do explicit hashtable probe if the total of nblocks of relation's forks
+	 * is not invalid and the nblocks to be invalidated is less than the
+	 * full-scan threshold of buffer pool.  Otherwise, full scan is executed.
+	 */
+	if (nTotalBlocks != InvalidBlockNumber &&
+		nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		curBlock;
+
+			nForkBlocks = smgrcachednblocks(smgr_reln, forkNum[j]);
+
+			for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks; curBlock++)
+			{
+				uint32		bufHash;		/* hash value for tag */
+				BufferTag	bufTag;			/* identity of requested block */
+				LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				bufHash = BufTableHashCode(&bufTag);
+				bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing. */
+				LWLockAcquire(bufPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&bufTag, bufHash);
+				LWLockRelease(bufPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= firstDelBlock[j])
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3244,8 +3326,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..c089e62 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -565,6 +565,29 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 }
 
 /*
+ *	smgrcachednblocks() -- Calculate the number of blocks that are cached in
+ *					 	   the supplied relation.
+ *
+ * It is equivalent to calling smgrnblocks, but only used in recovery for now.
+ * The returned value of file size could be inaccurate because the lseek of buggy
+ * Linux kernels might not have accounted the recent file extension or write.
+ * However, this function ensures that cached values are only used in recovery,
+ * since there is no shared invalidation mechanism that is implemented yet for
+ * changes in file size.
+ *
+ * This returns an InvalidBlockNumber when smgr_cached_nblocks is not available
+ * and when not in recovery.
+ */
+BlockNumber
+smgrcachednblocks(SMgrRelation reln, ForkNumber forknum)
+{
+	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+		return reln->smgr_cached_nblocks[forknum];
+
+	return InvalidBlockNumber;
+}
+
+/*
  *	smgrtruncate() -- Truncate the given forks of supplied relation to
  *					  each specified numbers of blocks
  *
@@ -583,7 +606,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..fbb7a4a 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -99,6 +99,7 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
 extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrcachednblocks(SMgrRelation reln, ForkNumber forknum);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchapplication/octet-stream; name=v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchDownload
From 6b0219580f34a3c0b63445521d6ba1fef3352f53 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH] Prevent invalidating blocks in smgrextend() during recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

#102Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#99)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Oct 2, 2020 at 8:14 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Thu, 1 Oct 2020 12:55:34 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in

On Thursday, October 1, 2020 4:52 PM, Tsunakawa-san wrote:

(I'm still mildly opposed to the function name, which seems exposing
detail too much.)

Do you have any better proposal? BTW, I am still not sure whether it
is a good idea to expose a new API for this especially because we do
exactly the same thing in existing function smgrnblocks. Why not just
add a new bool *cached parameter in smgrnblocks which will be set if
we return cached value? I understand that we need to change the code
wherever we call smgrnblocks or maybe even extensions if they call
this function but it is not clear to me if that is a big deal. What do
you think? I am not opposed to introducing the new API but I feel that
adding a new parameter to the existing API to handle this case is a
better option.

--
With Regards,
Amit Kapila.

#103Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#101)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Mon, Oct 5, 2020 at 6:59 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Friday, October 2, 2020 11:45 AM, Horiguchi-san wrote:

Thaks for the new version.

Thank you for your thoughtful reviews!
I've attached an updated patch addressing the comments below.

Few comments:
===============
1.
@@ -2990,10 +3002,80 @@ DropRelFileNodeBuffers(RelFileNodeBackend
rnode, ForkNumber *forkNum,
return;
}

+ /*
+ * Get the total number of cached blocks and to-be-invalidated blocks
+ * of the relation.  The cached value returned by smgrcachednblocks
+ * could be smaller than the actual number of existing buffers of the
+ * file.  This is caused by buggy Linux kernels that might not have
+ * accounted the recent write.  If a fork's nblocks is invalid, exit loop.
+ */
+ for (i = 0; i < nforks; i++)
+ {
+ /* Get the total nblocks for a relation's fork */
+ nForkBlocks = smgrcachednblocks(smgr_reln, forkNum[i]);
+
+ if (nForkBlocks == InvalidBlockNumber)
+ {
+ nTotalBlocks = InvalidBlockNumber;
+ break;
+ }
+ nTotalBlocks += nForkBlocks;
+ nBlocksToInvalidate = nTotalBlocks - firstDelBlock[i];
+ }
+
+ /*
+ * Do explicit hashtable probe if the total of nblocks of relation's forks
+ * is not invalid and the nblocks to be invalidated is less than the
+ * full-scan threshold of buffer pool.  Otherwise, full scan is executed.
+ */
+ if (nTotalBlocks != InvalidBlockNumber &&
+ nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+ {
+ for (j = 0; j < nforks; j++)
+ {
+ BlockNumber curBlock;
+
+ nForkBlocks = smgrcachednblocks(smgr_reln, forkNum[j]);
+
+ for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks; curBlock++)

What if one or more of the forks doesn't have cached value? I think
the patch will skip such forks and will invalidate/unpin buffers for
others. You probably need a local array of nForkBlocks which will be
formed first time and then used in the second loop. You also in some
way need to handle the case where that local array doesn't have cached
blocks.

2. Also, the other thing is I have asked for some testing to avoid the
small regression we have for a smaller number of shared buffers which
I don't see the results nor any change in the code. I think it is
better if you post the pending/open items each time you post a new
version of the patch.

--
With Regards,
Amit Kapila.

#104k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#103)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Monday, October 5, 2020 3:30 PM, Amit Kapila wrote:

+ for (i = 0; i < nforks; i++)
+ {
+ /* Get the total nblocks for a relation's fork */ nForkBlocks =
+ smgrcachednblocks(smgr_reln, forkNum[i]);
+
+ if (nForkBlocks == InvalidBlockNumber) { nTotalBlocks =
+ InvalidBlockNumber; break; } nTotalBlocks += nForkBlocks;
+ nBlocksToInvalidate = nTotalBlocks - firstDelBlock[i]; }
+
+ /*
+ * Do explicit hashtable probe if the total of nblocks of relation's
+ forks
+ * is not invalid and the nblocks to be invalidated is less than the
+ * full-scan threshold of buffer pool.  Otherwise, full scan is executed.
+ */
+ if (nTotalBlocks != InvalidBlockNumber && nBlocksToInvalidate <
+ BUF_DROP_FULL_SCAN_THRESHOLD) { for (j = 0; j < nforks; j++) {
+ BlockNumber curBlock;
+
+ nForkBlocks = smgrcachednblocks(smgr_reln, forkNum[j]);
+
+ for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks; curBlock++)

What if one or more of the forks doesn't have cached value? I think the patch
will skip such forks and will invalidate/unpin buffers for others.

Not having a cached value is equivalent to InvalidBlockNumber, right?
Maybe I'm missing something? But in the first loop we are already doing the
pre-check of whether or not one of the forks doesn't have cached value.
If it's not cached, then the nTotalBlocks is set to InvalidBlockNumber so we
won't need to enter the optimization loop and just execute the full scan buffer
invalidation process.

You probably
need a local array of nForkBlocks which will be formed first time and then
used in the second loop. You also in some way need to handle the case where
that local array doesn't have cached blocks.

Understood. that would be cleaner.
BlockNumber nForkBlocks[MAX_FORKNUM];

As for handling whether the local array is empty, I think the first loop would cover it,
and there's no need to pre-check if the array is empty again in the second loop.
for (i = 0; i < nforks; i++)
{
nForkBlocks[i] = smgrcachednblocks(smgr_reln, forkNum[i]);

if (nForkBlocks[i] == InvalidBlockNumber)
{
nTotalBlocks = InvalidBlockNumber;
break;
}
nTotalBlocks += nForkBlocks[i];
nBlocksToInvalidate = nTotalBlocks - firstDelBlock[i];
}

2. Also, the other thing is I have asked for some testing to avoid the small
regression we have for a smaller number of shared buffers which I don't see
the results nor any change in the code. I think it is better if you post the
pending/open items each time you post a new version of the patch.

Ah. Apologies for forgetting to include updates about that, but since I keep on updating
the patch I've decided not to post results yet as performance may vary per patch-update
due to possible bugs.
But for the performance case of not using recovery check, I just removed it from below.
Does it meet the intention?

BlockNumber smgrcachednblocks(SMgrRelation reln, ForkNumber forknum) {
-       if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+       if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
                return reln->smgr_cached_nblocks[forknum];

Regards,
Kirk Jamison

#105Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#104)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Mon, Oct 5, 2020 at 3:04 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Monday, October 5, 2020 3:30 PM, Amit Kapila wrote:

+ for (i = 0; i < nforks; i++)
+ {
+ /* Get the total nblocks for a relation's fork */ nForkBlocks =
+ smgrcachednblocks(smgr_reln, forkNum[i]);
+
+ if (nForkBlocks == InvalidBlockNumber) { nTotalBlocks =
+ InvalidBlockNumber; break; } nTotalBlocks += nForkBlocks;
+ nBlocksToInvalidate = nTotalBlocks - firstDelBlock[i]; }
+
+ /*
+ * Do explicit hashtable probe if the total of nblocks of relation's
+ forks
+ * is not invalid and the nblocks to be invalidated is less than the
+ * full-scan threshold of buffer pool.  Otherwise, full scan is executed.
+ */
+ if (nTotalBlocks != InvalidBlockNumber && nBlocksToInvalidate <
+ BUF_DROP_FULL_SCAN_THRESHOLD) { for (j = 0; j < nforks; j++) {
+ BlockNumber curBlock;
+
+ nForkBlocks = smgrcachednblocks(smgr_reln, forkNum[j]);
+
+ for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks; curBlock++)

What if one or more of the forks doesn't have cached value? I think the patch
will skip such forks and will invalidate/unpin buffers for others.

Not having a cached value is equivalent to InvalidBlockNumber, right?
Maybe I'm missing something? But in the first loop we are already doing the
pre-check of whether or not one of the forks doesn't have cached value.
If it's not cached, then the nTotalBlocks is set to InvalidBlockNumber so we
won't need to enter the optimization loop and just execute the full scan buffer
invalidation process.

oh, I have missed that, so the existing code will work fine for that case.

You probably
need a local array of nForkBlocks which will be formed first time and then
used in the second loop. You also in some way need to handle the case where
that local array doesn't have cached blocks.

Understood. that would be cleaner.
BlockNumber nForkBlocks[MAX_FORKNUM];

As for handling whether the local array is empty, I think the first loop would cover it,
and there's no need to pre-check if the array is empty again in the second loop.
for (i = 0; i < nforks; i++)
{
nForkBlocks[i] = smgrcachednblocks(smgr_reln, forkNum[i]);

if (nForkBlocks[i] == InvalidBlockNumber)
{
nTotalBlocks = InvalidBlockNumber;
break;
}
nTotalBlocks += nForkBlocks[i];
nBlocksToInvalidate = nTotalBlocks - firstDelBlock[i];
}

This appears okay.

2. Also, the other thing is I have asked for some testing to avoid the small
regression we have for a smaller number of shared buffers which I don't see
the results nor any change in the code. I think it is better if you post the
pending/open items each time you post a new version of the patch.

Ah. Apologies for forgetting to include updates about that, but since I keep on updating
the patch I've decided not to post results yet as performance may vary per patch-update
due to possible bugs.
But for the performance case of not using recovery check, I just removed it from below.
Does it meet the intention?

BlockNumber smgrcachednblocks(SMgrRelation reln, ForkNumber forknum) {
-       if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+       if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
return reln->smgr_cached_nblocks[forknum];

Yes, we can do that for the purpose of testing.

--
With Regards,
Amit Kapila.

#106k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#105)
3 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Monday, October 5, 2020 8:50 PM, Amit Kapila wrote:

On Mon, Oct 5, 2020 at 3:04 PM k.jamison@fujitsu.com

2. Also, the other thing is I have asked for some testing to avoid
the small regression we have for a smaller number of shared buffers
which I don't see the results nor any change in the code. I think it
is better if you post the pending/open items each time you post a new

version of the patch.

Ah. Apologies for forgetting to include updates about that, but since
I keep on updating the patch I've decided not to post results yet as
performance may vary per patch-update due to possible bugs.
But for the performance case of not using recovery check, I just removed it

from below.

Does it meet the intention?

BlockNumber smgrcachednblocks(SMgrRelation reln, ForkNumber

forknum) {

- if (InRecovery && reln->smgr_cached_nblocks[forknum] !=

InvalidBlockNumber)

+ if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
return reln->smgr_cached_nblocks[forknum];

Yes, we can do that for the purpose of testing.

With the latest patches attached, and removing the recovery check in smgrnblocks,
I tested the performance of vacuum.
(3 trial runs, 3.5 GB db populated with 1000 tables)

Execution Time (seconds)
| s_b | master | patched | %reg |
|-------|--------|---------|----------|
| 128MB | 15.265 | 15.260 | -0.03% |
| 1GB | 14.808 | 15.009 | 1.34% |
| 20GB | 24.673 | 11.681 | -111.22% |
| 100GB | 74.298 | 11.724 | -533.73% |

These are good results and we can see the improvements for large shared buffers,
For small s_b, the performance is almost the same.

I repeated the recovery performance test from the previous mail,
and ran three trials for each shared_buffer setting.
We can also clearly see the improvement here.

Recovery Time (seconds)
| s_b | master | patched | %reg |
|-------|--------|---------|--------|
| 128MB | 3.043 | 3.010 | -1.10% |
| 1GB | 3.417 | 3.477 | 1.73% |
| 20GB | 20.597 | 2.409 | -755% |
| 100GB | 66.862 | 2.409 | -2676% |

For default and small shared_buffers, the recovery performance is almost the same.
But for bigger shared_buffers, we can see the benefit and improvement.
For 20GB, from 20.597 s to 2.409 s. For 100GB s_b, from 66.862 s to 2.409 s.

I have updated the latest patches, with 0002 being the new one.
Instead of introducing a new API, I just added the bool parameter to smgrnblocks
and modified its callers.

Comments and feedback are highly appreciated.

Regards,
Kirk Jamison

Attachments:

0001-v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchapplication/octet-stream; name=0001-v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchDownload
From 859c547487109481f13ee36db8ca884e4475da2c Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH 1/3] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

0002-v1-Add-bool-param-in-arg-of-smgrextend-for-cached-block.patchapplication/octet-stream; name=0002-v1-Add-bool-param-in-arg-of-smgrextend-for-cached-block.patchDownload
From cec0f764901cf33ba66dca40eb2efa5f475076fb Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH 2/3] Add bool param in arg of smgrextend for cached blocks

---
 src/backend/access/gist/gistbuild.c       |  4 +++-
 src/backend/access/heap/visibilitymap.c   |  9 ++++++---
 src/backend/access/table/tableam.c        |  5 +++--
 src/backend/access/transam/xlogutils.c    |  3 ++-
 src/backend/catalog/storage.c             |  6 ++++--
 src/backend/storage/buffer/bufmgr.c       | 12 ++++++++----
 src/backend/storage/freespace/freespace.c |  9 ++++++---
 src/backend/storage/smgr/smgr.c           | 19 ++++++++++++++++++-
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 52 insertions(+), 18 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 28bc585..d599655 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -808,6 +808,7 @@ gistBuildCallback(Relation index,
 	GISTBuildState *buildstate = (GISTBuildState *) state;
 	IndexTuple	itup;
 	MemoryContext oldCtx;
+	bool		isCached;
 
 	oldCtx = MemoryContextSwitchTo(buildstate->giststate->tempCxt);
 
@@ -856,7 +857,8 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM,
+											&isCached)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..6d2be35 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -445,6 +445,7 @@ BlockNumber
 visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 {
 	BlockNumber newnblocks;
+	bool	cached;
 
 	/* last remaining block, byte, and bit */
 	BlockNumber truncBlock = HEAPBLK_TO_MAPBLOCK(nheapblocks);
@@ -528,7 +529,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, &cached) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -547,6 +548,7 @@ static Buffer
 vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 {
 	Buffer		buf;
+	bool		cached;
 
 	/*
 	 * We might not have opened the relation at the smgr level yet, or we
@@ -564,7 +566,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, &cached);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -618,6 +620,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 {
 	BlockNumber vm_nblocks_now;
 	PGAlignedBlock pg;
+	bool	cached;
 
 	PageInit((Page) pg.data, BLCKSZ, 0);
 
@@ -647,7 +650,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, &cached);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..e64451f 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -628,6 +628,7 @@ uint64
 table_block_relation_size(Relation rel, ForkNumber forkNumber)
 {
 	uint64		nblocks = 0;
+	bool		cached;
 
 	/* Open it at the smgr level if not already done */
 	RelationOpenSmgr(rel);
@@ -636,10 +637,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, &cached);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, &cached);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 7e915bc..20eda81 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -444,6 +444,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	BlockNumber lastblock;
 	Buffer		buffer;
 	SMgrRelation smgr;
+	bool		cached;
 
 	Assert(blkno != P_NEW);
 
@@ -460,7 +461,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, &cached);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index dbbd3aa..057e53b 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -410,6 +410,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 {
 	PGAlignedBlock buf;
 	Page		page;
+	bool		cached;
 	bool		use_wal;
 	bool		copying_initfork;
 	BlockNumber nblocks;
@@ -434,7 +435,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, &cached);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -703,6 +704,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 		BlockNumber nblocks[MAX_FORKNUM + 1];
 		BlockNumber total_blocks = 0;
 		SMgrRelation srel;
+		bool		cached;
 
 		srel = smgropen(pendingsync->rnode, InvalidBackendId);
 
@@ -720,7 +722,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, &cached);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..45897a3 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -720,6 +720,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 	BufferDesc *bufHdr;
 	Block		bufBlock;
 	bool		found;
+	bool		isCached;
 	bool		isExtend;
 	bool		isLocalBuf = SmgrIsTemp(smgr);
 
@@ -739,7 +740,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, &isCached);
 
 	if (isLocalBuf)
 	{
@@ -2852,11 +2853,14 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 		case RELKIND_SEQUENCE:
 		case RELKIND_INDEX:
 		case RELKIND_PARTITIONED_INDEX:
-			/* Open it at the smgr level if not already done */
-			RelationOpenSmgr(relation);
+			{
+				bool		isCached;
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+				/* Open it at the smgr level if not already done */
+				RelationOpenSmgr(relation);
 
+				return smgrnblocks(relation->rd_smgr, forkNum, &isCached);
+			}
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
 		case RELKIND_MATVIEW:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..f631878 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -264,6 +264,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	FSMAddress	first_removed_address;
 	uint16		first_removed_slot;
 	Buffer		buf;
+	bool		cached;
 
 	RelationOpenSmgr(rel);
 
@@ -317,7 +318,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, &cached) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -532,6 +533,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 {
 	BlockNumber blkno = fsm_logical_to_physical(addr);
 	Buffer		buf;
+	bool		cached;
 
 	RelationOpenSmgr(rel);
 
@@ -547,7 +549,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, &cached);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -603,6 +605,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 {
 	BlockNumber fsm_nblocks_now;
 	PGAlignedBlock pg;
+	bool	cached;
 
 	PageInit((Page) pg.data, BLCKSZ, 0);
 
@@ -633,7 +636,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, &cached);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..6e2b680 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -551,23 +551,40 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 /*
  *	smgrnblocks() -- Calculate the number of blocks in the
  *					 supplied relation.
+ *
+ * This returns an InvalidBlockNumber when smgr_cached_nblocks is not
+ * available and when not in recovery path.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *isCached)
 {
 	BlockNumber result;
 
+	*isCached = false;
+
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
 	 * invalidation mechanism for changes in file size.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		*isCached = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
 	reln->smgr_cached_nblocks[forknum] = result;
 
+	/*
+	 * We cannot believe the result from smgr_nblocks is always accurate
+	 * because lseek of buggy Linux kernels doesn't account for a recent
+	 * write.
+	 */
+	if (!InRecovery && result == InvalidBlockNumber)
+		return InvalidBlockNumber;
+
 	return result;
 }
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..e247cce 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *isCached);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

0003-v2-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=0003-v2-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From c69409f04e486449971b1f8eb3235429eb2cf969 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 10:23:51 +0000
Subject: [PATCH 3/3] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 98 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     |  2 +-
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 91 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 45897a3..fae9980 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2969,18 +2971,29 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are known to be accurate. See smgrnblocks() for
+ *		more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		isCached;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+	BlockNumber	nTotalBlocks = 0;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2994,10 +3007,78 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of cached blocks and to-be-invalidated blocks
+	 * of the relation.  The cached value returned by smgrnblocks could be
+	 * smaller than the actual number of existing buffers of the file.
+	 * This is caused by buggy Linux kernels that might not have accounted
+	 * the recent write.  If a fork's nblocks is invalid, exit loop.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the total nblocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &isCached);
+
+		if (nForkBlocks[i] == InvalidBlockNumber)
+		{
+			nTotalBlocks = InvalidBlockNumber;
+			break;
+		}
+		nTotalBlocks += nForkBlocks[i];
+		nBlocksToInvalidate = nTotalBlocks - firstDelBlock[i];
+	}
+
+	/*
+	 * Do explicit hashtable lookup if the total of nblocks of relation's forks
+	 * is not invalid and the nblocks to be invalidated is less than the
+	 * full-scan threshold of buffer pool.  Otherwise, full scan is executed.
+	 */
+	if (nTotalBlocks != InvalidBlockNumber &&
+		nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		curBlock;
+
+			for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j]; curBlock++)
+			{
+				uint32		bufHash;		/* hash value for tag */
+				BufferTag	bufTag;			/* identity of requested block */
+				LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				bufHash = BufTableHashCode(&bufTag);
+				bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing. */
+				LWLockAcquire(bufPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&bufTag, bufHash);
+				LWLockRelease(bufPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= firstDelBlock[j])
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3248,8 +3329,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 6e2b680..8e189db 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -607,7 +607,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

#107tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#106)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

With the latest patches attached, and removing the recovery check in
smgrnblocks, I tested the performance of vacuum.
(3 trial runs, 3.5 GB db populated with 1000 tables)

Execution Time (seconds)
| s_b | master | patched | %reg |
|-------|--------|---------|----------|
| 128MB | 15.265 | 15.260 | -0.03% |
| 1GB | 14.808 | 15.009 | 1.34% |
| 20GB | 24.673 | 11.681 | -111.22% | 100GB | 74.298 | 11.724 |
| -533.73% |

These are good results and we can see the improvements for large shared
buffers, For small s_b, the performance is almost the same.

Very nice!

I'll try to review the patch again soon.

Regards
Takayuki Tsunakawa

#108tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#106)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi Kirk san,

(1)
+ * This returns an InvalidBlockNumber when smgr_cached_nblocks is not
+ * available and when not in recovery path.
+	/*
+	 * We cannot believe the result from smgr_nblocks is always accurate
+	 * because lseek of buggy Linux kernels doesn't account for a recent
+	 * write.
+	 */
+	if (!InRecovery && result == InvalidBlockNumber)
+		return InvalidBlockNumber;
+

These are unnecessary, because mdnblocks() never returns InvalidBlockNumber and conseuently smgrnblocks() doesn't return InvalidBlockNumber.

(2)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *isCached)

I think it's better to make the argument name iscached so that camel case aligns with forknum, which is not forkNum.

(3)
+	 * This is caused by buggy Linux kernels that might not have accounted
+	 * the recent write.  If a fork's nblocks is invalid, exit loop.

"accounted for" is the right English?
I think The second sentence should be described in terms of its meaning, not the program logic. For example, something like "Give up the optimization if the block count of any fork cannot be trusted."
Likewise, express the following part in semantics.

+	 * Do explicit hashtable lookup if the total of nblocks of relation's forks
+	 * is not invalid and the nblocks to be invalidated is less than the
(4)
+		if (nForkBlocks[i] == InvalidBlockNumber)
+		{
+			nTotalBlocks = InvalidBlockNumber;
+			break;
+		}

Use isCached in if condition because smgrnblocks() doesn't return InvalidBlockNumber.

(5)
+ nBlocksToInvalidate = nTotalBlocks - firstDelBlock[i];

should be

+ nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);

(6)
+					bufHdr->tag.blockNum >= firstDelBlock[j])
+					InvalidateBuffer(bufHdr);	/* releases spinlock */

The right side of >= should be cur_block.

Regards
Takayuki Tsunakawa

#109k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#108)
3 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, October 8, 2020 3:38 PM, Tsunakawa-san wrote:

Hi Kirk san,

Thank you for looking into my patches!

(1)
+ * This returns an InvalidBlockNumber when smgr_cached_nblocks is not
+ * available and when not in recovery path.
+	/*
+	 * We cannot believe the result from smgr_nblocks is always accurate
+	 * because lseek of buggy Linux kernels doesn't account for a recent
+	 * write.
+	 */
+	if (!InRecovery && result == InvalidBlockNumber)
+		return InvalidBlockNumber;
+

These are unnecessary, because mdnblocks() never returns
InvalidBlockNumber and conseuently smgrnblocks() doesn't return
InvalidBlockNumber.

Yes. Thanks for carefully looking into that. Removed.

(2)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *isCached)

I think it's better to make the argument name iscached so that camel case
aligns with forknum, which is not forkNum.

This is kinda tricky because of the surrounding code which follows inconsistent coding style too.
So I just followed the same like below and retained the change.

extern void smgrcreate(SMgrRelation reln, ForkNumber forknum, bool isRedo);
extern void smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, char *buffer, bool skipFsync);

(3)
+	 * This is caused by buggy Linux kernels that might not have
accounted
+	 * the recent write.  If a fork's nblocks is invalid, exit loop.

"accounted for" is the right English?
I think The second sentence should be described in terms of its meaning, not
the program logic. For example, something like "Give up the optimization if
the block count of any fork cannot be trusted."

Fixed.

Likewise, express the following part in semantics.

+	 * Do explicit hashtable lookup if the total of nblocks of relation's
forks
+	 * is not invalid and the nblocks to be invalidated is less than the

I revised it like below:
"Look up the buffer in the hashtable if the block size is known to
be accurate and the total blocks to be invalidated is below the
full scan threshold. Otherwise, give up the optimization."

(4)
+		if (nForkBlocks[i] == InvalidBlockNumber)
+		{
+			nTotalBlocks = InvalidBlockNumber;
+			break;
+		}

Use isCached in if condition because smgrnblocks() doesn't return
InvalidBlockNumber.

Fixed. if (!isCached)

(5)
+ nBlocksToInvalidate = nTotalBlocks - firstDelBlock[i];

should be

+ nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);

Fixed.

(6)
+ bufHdr->tag.blockNum >=
firstDelBlock[j])
+ InvalidateBuffer(bufHdr); /*
releases spinlock */

The right side of >= should be cur_block.

Fixed.

Attached are the updated patches.
Thank you again for the reviews.

Regards,
Kirk Jamison

Attachments:

0001-v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchapplication/octet-stream; name=0001-v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchDownload
From f357f48100c3856a173f8644db6e5c7fb90b5f9f Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH 1/3] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

0002-v2-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=0002-v2-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From 5b5bccc7cb6cd7b55a5c244d9ab1e73c746a8cca Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH 2/3] Add bool param in smgrnblocks() for cached blocks.

---
 src/backend/access/gist/gistbuild.c       |  4 +++-
 src/backend/access/heap/visibilitymap.c   |  9 ++++++---
 src/backend/access/table/tableam.c        |  5 +++--
 src/backend/access/transam/xlogutils.c    |  3 ++-
 src/backend/catalog/storage.c             |  6 ++++--
 src/backend/storage/buffer/bufmgr.c       | 12 ++++++++----
 src/backend/storage/freespace/freespace.c |  9 ++++++---
 src/backend/storage/smgr/smgr.c           | 13 +++++++++++--
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 45 insertions(+), 19 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 28bc585..d599655 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -808,6 +808,7 @@ gistBuildCallback(Relation index,
 	GISTBuildState *buildstate = (GISTBuildState *) state;
 	IndexTuple	itup;
 	MemoryContext oldCtx;
+	bool		isCached;
 
 	oldCtx = MemoryContextSwitchTo(buildstate->giststate->tempCxt);
 
@@ -856,7 +857,8 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM,
+											&isCached)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..6d2be35 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -445,6 +445,7 @@ BlockNumber
 visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 {
 	BlockNumber newnblocks;
+	bool	cached;
 
 	/* last remaining block, byte, and bit */
 	BlockNumber truncBlock = HEAPBLK_TO_MAPBLOCK(nheapblocks);
@@ -528,7 +529,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, &cached) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -547,6 +548,7 @@ static Buffer
 vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 {
 	Buffer		buf;
+	bool		cached;
 
 	/*
 	 * We might not have opened the relation at the smgr level yet, or we
@@ -564,7 +566,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, &cached);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -618,6 +620,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 {
 	BlockNumber vm_nblocks_now;
 	PGAlignedBlock pg;
+	bool	cached;
 
 	PageInit((Page) pg.data, BLCKSZ, 0);
 
@@ -647,7 +650,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, &cached);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..e64451f 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -628,6 +628,7 @@ uint64
 table_block_relation_size(Relation rel, ForkNumber forkNumber)
 {
 	uint64		nblocks = 0;
+	bool		cached;
 
 	/* Open it at the smgr level if not already done */
 	RelationOpenSmgr(rel);
@@ -636,10 +637,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, &cached);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, &cached);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 7e915bc..20eda81 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -444,6 +444,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	BlockNumber lastblock;
 	Buffer		buffer;
 	SMgrRelation smgr;
+	bool		cached;
 
 	Assert(blkno != P_NEW);
 
@@ -460,7 +461,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, &cached);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index dbbd3aa..057e53b 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -410,6 +410,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 {
 	PGAlignedBlock buf;
 	Page		page;
+	bool		cached;
 	bool		use_wal;
 	bool		copying_initfork;
 	BlockNumber nblocks;
@@ -434,7 +435,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, &cached);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -703,6 +704,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 		BlockNumber nblocks[MAX_FORKNUM + 1];
 		BlockNumber total_blocks = 0;
 		SMgrRelation srel;
+		bool		cached;
 
 		srel = smgropen(pendingsync->rnode, InvalidBackendId);
 
@@ -720,7 +722,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, &cached);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..45897a3 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -720,6 +720,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 	BufferDesc *bufHdr;
 	Block		bufBlock;
 	bool		found;
+	bool		isCached;
 	bool		isExtend;
 	bool		isLocalBuf = SmgrIsTemp(smgr);
 
@@ -739,7 +740,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, &isCached);
 
 	if (isLocalBuf)
 	{
@@ -2852,11 +2853,14 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 		case RELKIND_SEQUENCE:
 		case RELKIND_INDEX:
 		case RELKIND_PARTITIONED_INDEX:
-			/* Open it at the smgr level if not already done */
-			RelationOpenSmgr(relation);
+			{
+				bool		isCached;
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+				/* Open it at the smgr level if not already done */
+				RelationOpenSmgr(relation);
 
+				return smgrnblocks(relation->rd_smgr, forkNum, &isCached);
+			}
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
 		case RELKIND_MATVIEW:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..f631878 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -264,6 +264,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	FSMAddress	first_removed_address;
 	uint16		first_removed_slot;
 	Buffer		buf;
+	bool		cached;
 
 	RelationOpenSmgr(rel);
 
@@ -317,7 +318,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, &cached) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -532,6 +533,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 {
 	BlockNumber blkno = fsm_logical_to_physical(addr);
 	Buffer		buf;
+	bool		cached;
 
 	RelationOpenSmgr(rel);
 
@@ -547,7 +549,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, &cached);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -603,6 +605,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 {
 	BlockNumber fsm_nblocks_now;
 	PGAlignedBlock pg;
+	bool	cached;
 
 	PageInit((Page) pg.data, BLCKSZ, 0);
 
@@ -633,7 +636,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, &cached);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..d7d61ea 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -553,16 +553,25 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
  *					 supplied relation.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *isCached)
 {
 	BlockNumber result;
 
+	*isCached = false;
+
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  The cached values
+	 * could be smaller than the actual number of existing buffers of the file.
+	 * This is caused by lseek of buggy Linux kernels that might not have
+	 * accounted for the recent write.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		*isCached = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..e247cce 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *isCached);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

0003-v23-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=0003-v23-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 6ca5c03d6df13adae64a7bc0f3dc4520a4c110d8 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Thu, 8 Oct 2020 08:46:43 +0000
Subject: [PATCH 3/3] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 99 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     |  2 +-
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 92 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 45897a3..507b373 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2969,18 +2971,29 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are known to be accurate. See smgrnblocks() for
+ *		more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		isCached;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+	BlockNumber	nTotalBlocks = 0;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2994,10 +3007,79 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of cached blocks and to-be-invalidated blocks
+	 * of the relation.  The cached value returned by smgrnblocks could be
+	 * smaller than the actual number of existing buffers of the file.
+	 * This is caused by buggy Linux kernels that might not have accounted
+	 * for the recent write.  Give up the optimization if the block count
+	 * of any fork cannot be trusted.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the total nblocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &isCached);
+
+		if (!isCached)
+		{
+			nTotalBlocks = InvalidBlockNumber;
+			break;
+		}
+		nTotalBlocks += nForkBlocks[i];
+		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffer in the hashtable if the block size is known to
+	 * be accurate and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (nTotalBlocks != InvalidBlockNumber &&
+		nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		curBlock;
+
+			for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j]; curBlock++)
+			{
+				uint32		bufHash;		/* hash value for tag */
+				BufferTag	bufTag;			/* identity of requested block */
+				LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				bufHash = BufTableHashCode(&bufTag);
+				bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing. */
+				LWLockAcquire(bufPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&bufTag, bufHash);
+				LWLockRelease(bufPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= cur_block)
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3248,8 +3330,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index d7d61ea..6253c13 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -599,7 +599,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

#110k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#109)
3 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi,

Attached are the updated patches.

Sorry there was an error in the 3rd patch. So attached is a rebase one.

Regards,
Kirk Jamison

Attachments:

0001-v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchapplication/octet-stream; name=0001-v1-Prevent-invalidating-blocks-in-smgrextend-during-recovery.patchDownload
From f357f48100c3856a173f8644db6e5c7fb90b5f9f Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH 1/3] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

0002-v2-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=0002-v2-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From 5b5bccc7cb6cd7b55a5c244d9ab1e73c746a8cca Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH 2/3] Add bool param in smgrnblocks() for cached blocks.

---
 src/backend/access/gist/gistbuild.c       |  4 +++-
 src/backend/access/heap/visibilitymap.c   |  9 ++++++---
 src/backend/access/table/tableam.c        |  5 +++--
 src/backend/access/transam/xlogutils.c    |  3 ++-
 src/backend/catalog/storage.c             |  6 ++++--
 src/backend/storage/buffer/bufmgr.c       | 12 ++++++++----
 src/backend/storage/freespace/freespace.c |  9 ++++++---
 src/backend/storage/smgr/smgr.c           | 13 +++++++++++--
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 45 insertions(+), 19 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 28bc585..d599655 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -808,6 +808,7 @@ gistBuildCallback(Relation index,
 	GISTBuildState *buildstate = (GISTBuildState *) state;
 	IndexTuple	itup;
 	MemoryContext oldCtx;
+	bool		isCached;
 
 	oldCtx = MemoryContextSwitchTo(buildstate->giststate->tempCxt);
 
@@ -856,7 +857,8 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM,
+											&isCached)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..6d2be35 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -445,6 +445,7 @@ BlockNumber
 visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 {
 	BlockNumber newnblocks;
+	bool	cached;
 
 	/* last remaining block, byte, and bit */
 	BlockNumber truncBlock = HEAPBLK_TO_MAPBLOCK(nheapblocks);
@@ -528,7 +529,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, &cached) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -547,6 +548,7 @@ static Buffer
 vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 {
 	Buffer		buf;
+	bool		cached;
 
 	/*
 	 * We might not have opened the relation at the smgr level yet, or we
@@ -564,7 +566,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, &cached);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -618,6 +620,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 {
 	BlockNumber vm_nblocks_now;
 	PGAlignedBlock pg;
+	bool	cached;
 
 	PageInit((Page) pg.data, BLCKSZ, 0);
 
@@ -647,7 +650,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, &cached);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..e64451f 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -628,6 +628,7 @@ uint64
 table_block_relation_size(Relation rel, ForkNumber forkNumber)
 {
 	uint64		nblocks = 0;
+	bool		cached;
 
 	/* Open it at the smgr level if not already done */
 	RelationOpenSmgr(rel);
@@ -636,10 +637,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, &cached);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, &cached);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 7e915bc..20eda81 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -444,6 +444,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	BlockNumber lastblock;
 	Buffer		buffer;
 	SMgrRelation smgr;
+	bool		cached;
 
 	Assert(blkno != P_NEW);
 
@@ -460,7 +461,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, &cached);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index dbbd3aa..057e53b 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -410,6 +410,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 {
 	PGAlignedBlock buf;
 	Page		page;
+	bool		cached;
 	bool		use_wal;
 	bool		copying_initfork;
 	BlockNumber nblocks;
@@ -434,7 +435,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, &cached);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -703,6 +704,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 		BlockNumber nblocks[MAX_FORKNUM + 1];
 		BlockNumber total_blocks = 0;
 		SMgrRelation srel;
+		bool		cached;
 
 		srel = smgropen(pendingsync->rnode, InvalidBackendId);
 
@@ -720,7 +722,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, &cached);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..45897a3 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -720,6 +720,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 	BufferDesc *bufHdr;
 	Block		bufBlock;
 	bool		found;
+	bool		isCached;
 	bool		isExtend;
 	bool		isLocalBuf = SmgrIsTemp(smgr);
 
@@ -739,7 +740,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, &isCached);
 
 	if (isLocalBuf)
 	{
@@ -2852,11 +2853,14 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 		case RELKIND_SEQUENCE:
 		case RELKIND_INDEX:
 		case RELKIND_PARTITIONED_INDEX:
-			/* Open it at the smgr level if not already done */
-			RelationOpenSmgr(relation);
+			{
+				bool		isCached;
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+				/* Open it at the smgr level if not already done */
+				RelationOpenSmgr(relation);
 
+				return smgrnblocks(relation->rd_smgr, forkNum, &isCached);
+			}
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
 		case RELKIND_MATVIEW:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..f631878 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -264,6 +264,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	FSMAddress	first_removed_address;
 	uint16		first_removed_slot;
 	Buffer		buf;
+	bool		cached;
 
 	RelationOpenSmgr(rel);
 
@@ -317,7 +318,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, &cached) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -532,6 +533,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 {
 	BlockNumber blkno = fsm_logical_to_physical(addr);
 	Buffer		buf;
+	bool		cached;
 
 	RelationOpenSmgr(rel);
 
@@ -547,7 +549,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, &cached);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -603,6 +605,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 {
 	BlockNumber fsm_nblocks_now;
 	PGAlignedBlock pg;
+	bool	cached;
 
 	PageInit((Page) pg.data, BLCKSZ, 0);
 
@@ -633,7 +636,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, &cached);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..d7d61ea 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -553,16 +553,25 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
  *					 supplied relation.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *isCached)
 {
 	BlockNumber result;
 
+	*isCached = false;
+
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  The cached values
+	 * could be smaller than the actual number of existing buffers of the file.
+	 * This is caused by lseek of buggy Linux kernels that might not have
+	 * accounted for the recent write.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		*isCached = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..e247cce 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *isCached);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

0003-v23-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=0003-v23-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From ed50ec8abe7ae01fcb2c70f849e8e5dd808efdd6 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Thu, 8 Oct 2020 08:46:43 +0000
Subject: [PATCH 3/3] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 99 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     |  2 +-
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 92 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 45897a3..8798ad0 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2969,18 +2971,29 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are known to be accurate. See smgrnblocks() for
+ *		more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		isCached;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+	BlockNumber	nTotalBlocks = 0;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2994,10 +3007,79 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of cached blocks and to-be-invalidated blocks
+	 * of the relation.  The cached value returned by smgrnblocks could be
+	 * smaller than the actual number of existing buffers of the file.
+	 * This is caused by buggy Linux kernels that might not have accounted
+	 * for the recent write.  Give up the optimization if the block count
+	 * of any fork cannot be trusted.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the total nblocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &isCached);
+
+		if (!isCached)
+		{
+			nTotalBlocks = InvalidBlockNumber;
+			break;
+		}
+		nTotalBlocks += nForkBlocks[i];
+		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffer in the hashtable if the block size is known to
+	 * be accurate and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (nTotalBlocks != InvalidBlockNumber &&
+		nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		curBlock;
+
+			for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j]; curBlock++)
+			{
+				uint32		bufHash;		/* hash value for tag */
+				BufferTag	bufTag;			/* identity of requested block */
+				LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				bufHash = BufTableHashCode(&bufTag);
+				bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing. */
+				LWLockAcquire(bufPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&bufTag, bufHash);
+				LWLockRelease(bufPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= curBlock)
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3248,8 +3330,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index d7d61ea..6253c13 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -599,7 +599,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

#111tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#109)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

(6)
+ bufHdr->tag.blockNum >=
firstDelBlock[j])
+ InvalidateBuffer(bufHdr); /*
releases spinlock */

The right side of >= should be cur_block.

Fixed.

= should be =, shouldn't it?

Please measure and let us see just the recovery performance again because the critical part of the patch was modified. If the performance is good as the previous one, and there's no review interaction with others in progress, I'll mark the patch as ready for committer in a few days.

Regards
Takayuki Tsunakawa

#112Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#111)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Fri, 9 Oct 2020 00:41:24 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

(6)
+ bufHdr->tag.blockNum >=
firstDelBlock[j])
+ InvalidateBuffer(bufHdr); /*
releases spinlock */

The right side of >= should be cur_block.

Fixed.

= should be =, shouldn't it?

Please measure and let us see just the recovery performance again because the critical part of the patch was modified. If the performance is good as the previous one, and there's no review interaction with others in progress, I'll mark the patch as ready for committer in a few days.

The performance is expected to be kept since smgrnblocks() is called
in a non-hot code path and actually it is called at most four times
per a buffer drop in this patch. But it's better making it sure.

I have some comments on the latest patch.

@@ -445,6 +445,7 @@ BlockNumber
visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
{
BlockNumber newnblocks;
+ bool cached;

All the added variables added by 0002 is useless because all the
caller sites are not interested in the value. smgrnblocks should
accept NULL as isCached. (I'm agree with Tsunakawa-san that the
camel-case name is not common there.)

+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &isCached);
+
+		if (!isCached)

"is cached" is not the property that code is interested in. No other callers to smgrnblocks are interested in that property. The need for caching is purely internal of smgrnblocks().

On the other hand, we are going to utilize the property of "accuracy"
that is a biproduct of reducing fseek calls, and, again, not
interested in how it is achieved.

So I suggest that the name should be "accurite" or something that is
not suggest the mechanism used under the hood.

+	if (nTotalBlocks != InvalidBlockNumber &&
+		nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)

I don't think nTotalBlocks is useful. What we need here is only total
blocks for every forks (nForkBlocks[]) and the total number of buffers
to be invalidated for all forks (nBlocksToInvalidate).

The right side of >= should be cur_block.

Fixed.

= should be =, shouldn't it?

It's just from a paranoia. What we are going to invalidate is blocks
blockNum of which >= curBlock. Although actually there's no chance of
any other processes having replaced the buffer with another page (with
lower blockid) of the same relation after BugTableLookup(), that
condition makes it sure not to leave blocks to be invalidated left
alone.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#113Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Kyotaro Horiguchi (#112)
Re: [Patch] Optimize dropping of relation buffers using dlist

Oops! Sorry for the mistake.

At Fri, 09 Oct 2020 11:12:01 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

At Fri, 9 Oct 2020 00:41:24 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

(6)
+ bufHdr->tag.blockNum >=
firstDelBlock[j])
+ InvalidateBuffer(bufHdr); /*
releases spinlock */

The right side of >= should be cur_block.

Fixed.

= should be =, shouldn't it?

Please measure and let us see just the recovery performance again because the critical part of the patch was modified. If the performance is good as the previous one, and there's no review interaction with others in progress, I'll mark the patch as ready for committer in a few days.

The performance is expected to be kept since smgrnblocks() is called
in a non-hot code path and actually it is called at most four times
per a buffer drop in this patch. But it's better making it sure.

I have some comments on the latest patch.

@@ -445,6 +445,7 @@ BlockNumber
visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
{
BlockNumber newnblocks;
+ bool cached;

All the added variables added by 0002 is useless because all the
caller sites are not interested in the value. smgrnblocks should
accept NULL as isCached. (I'm agree with Tsunakawa-san that the
camel-case name is not common there.)

+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &isCached);
+
+		if (!isCached)

"is cached" is not the property that code is interested in. No other callers to smgrnblocks are interested in that property. The need for caching is purely internal of smgrnblocks().

On the other hand, we are going to utilize the property of "accuracy"
that is a biproduct of reducing fseek calls, and, again, not
interested in how it is achieved.

So I suggest that the name should be "accurite" or something that is
not suggest the mechanism used under the hood.

+	if (nTotalBlocks != InvalidBlockNumber &&
+		nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)

I don't think nTotalBlocks is useful. What we need here is only total
blocks for every forks (nForkBlocks[]) and the total number of buffers
to be invalidated for all forks (nBlocksToInvalidate).

The right side of >= should be cur_block.

Fixed.

= should be =, shouldn't it?

It's just from a paranoia. What we are going to invalidate is blocks
blockNum of which >= curBlock. Although actually there's no chance of

Sorry. What we are going to invalidate is blocks that are blocNum >=
firstDelBlock[i]. So what I wanted to suggest was the condition should
be

+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= firstDelBlock[j])

any other processes having replaced the buffer with another page (with
lower blockid) of the same relation after BugTableLookup(), that
condition makes it sure not to leave blocks to be invalidated left
alone.

And I forgot to mention the patch names. I think many of us name the
patches using -v option of git-format-patch, and assign the version to
a patch-set thus the version number of all files that are posted at
once is same.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#114k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Kyotaro Horiguchi (#112)
3 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Friday, October 9, 2020 11:12 AM, Horiguchi-san wrote:

I have some comments on the latest patch.

Thank you for the feedback!
I've attached the latest patches.

visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks) {
BlockNumber newnblocks;
+ bool cached;

All the added variables added by 0002 is useless because all the caller sites
are not interested in the value. smgrnblocks should accept NULL as isCached.
(I'm agree with Tsunakawa-san that the camel-case name is not common
there.)

+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i],
&isCached);
+
+		if (!isCached)

"is cached" is not the property that code is interested in. No other callers to
smgrnblocks are interested in that property. The need for caching is purely
internal of smgrnblocks().
On the other hand, we are going to utilize the property of "accuracy"
that is a biproduct of reducing fseek calls, and, again, not interested in how it
is achieved.
So I suggest that the name should be "accurite" or something that is not
suggest the mechanism used under the hood.

I changed the bool param to "accurate" per your suggestion.
And I also removed the additional variables "bool cached" from the modified functions.
Now NULL values are accepted for the new boolean parameter

+	if (nTotalBlocks != InvalidBlockNumber &&
+		nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD)

I don't think nTotalBlocks is useful. What we need here is only total blocks for
every forks (nForkBlocks[]) and the total number of buffers to be invalidated
for all forks (nBlocksToInvalidate).

Alright. I also removed nTotalBlocks in v24-0003 patch.

for (i = 0; i < nforks; i++)
{
if (nForkBlocks[i] != InvalidBlockNumber &&
nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
{
Optimization loop
}
else
break;
}
if (i >= nforks)
return;
{ usual buffer invalidation process }

The right side of >= should be cur_block.

Fixed.
= should be =, shouldn't it?

It's just from a paranoia. What we are going to invalidate is blocks blockNum
of which >= curBlock. Although actually there's no chance of any other
processes having replaced the buffer with another page (with lower blockid)
of the same relation after BufTableLookup(), that condition makes it sure not
to leave blocks to be invalidated left alone.
Sorry. What we are going to invalidate is blocks that are blocNum >=
firstDelBlock[i]. So what I wanted to suggest was the condition should be

+				if (RelFileNodeEquals(bufHdr->tag.rnode,
rnode.node) &&
+					bufHdr->tag.forkNum ==
forkNum[j] &&
+					bufHdr->tag.blockNum >=
firstDelBlock[j])

I used bufHdr->tag.blockNum >= firstDelBlock[i] in the latest patch.

Please measure and let us see just the recovery performance again because

the critical part of the patch was modified. If the performance is good as the
previous one, and there's no review interaction with others in progress, I'll
mark the patch as ready for committer in a few days.

The performance is expected to be kept since smgrnblocks() is called in a
non-hot code path and actually it is called at most four times per a buffer
drop in this patch. But it's better making it sure.

Hmm. When I repeated the performance measurement for non-recovery,
I got almost similar execution results for both master and patched.

Execution Time (in seconds)
| s_b | master | patched | %reg |
|-------|--------|---------|--------|
| 128MB | 15.265 | 14.769 | -3.36% |
| 1GB | 14.808 | 14.618 | -1.30% |
| 20GB | 24.673 | 24.425 | -1.02% |
| 100GB | 74.298 | 74.813 | 0.69% |

That is considering that I removed the recovery-related checks in the patch and just
executed the commands on a standalone server.
-       if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+       if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)

OTOH, I also measured the recovery performance by having hot standby and executing failover.
The results were good and almost similar to the previously reported recovery performance.

Recovery Time (in seconds)
| s_b | master | patched | %reg |
|-------|--------|---------|--------|
| 128MB | 3.043 | 2.977 | -2.22% |
| 1GB | 3.417 | 3.41 | -0.21% |
| 20GB | 20.597 | 2.409 | -755% |
| 100GB | 66.862 | 2.409 | -2676% |

For 20GB s_b, from 20.597 s (Master) to 2.409 s (Patched).
For 100GB s_b, from 66.862 s (Master) to 2.409 s (Patched).
This is mainly benefits for large shared_buffers setting,
without compromising when shared_buffers is set to default or lower value.

If you could take a look again and if you have additional feedback or comments, I'd appreciate it.
Thank you for your time

Regards,
Kirk Jamison

Attachments:

v24-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v24-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From 664bcebd417ca9f470d1ad4dfe960e86f9d26113 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v24 1/3] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v24-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v24-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From b523cdb0bdc4eddbb0d0061165d64320ec2ec20c Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v24 2/3] Add bool param in smgrnblocks() for cached blocks.

---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 12 ++++++++++--
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 26 insertions(+), 17 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 28bc585..cfd46d6 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -856,7 +856,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 7e915bc..46fd97d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index dbbd3aa..4dc2ba8 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -720,7 +720,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..ae001fe 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -739,7 +739,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2855,7 +2855,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..a2577cb 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -553,16 +553,24 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
  *					 supplied relation.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  The cached values
+	 * could be smaller than the actual number of existing buffers of the file.
+	 * This is caused by lseek of buggy Linux kernels that might not have
+	 * accounted for the recent write.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (accurate != NULL)
+			*accurate = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..af11b53 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *accurate);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v24-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v24-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From a79d587593b173cf2868aca424a9e069aec88a77 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 9 Oct 2020 08:59:55 +0000
Subject: [PATCH v24 3/3] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 101 ++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 94 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ae001fe..5b37f9f 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2965,18 +2967,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are known to be accurate. See smgrnblocks() for
+ *		more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		accurate;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2990,10 +3002,82 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of cached blocks and to-be-invalidated blocks
+	 * of the relation.  The cached value returned by smgrnblocks could be
+	 * smaller than the actual number of existing buffers of the file.
+	 * This is caused by buggy Linux kernels that might not have accounted
+	 * for the recent write.  Give up the optimization if the block count
+	 * of any fork cannot be trusted.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the total nblocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &accurate);
+
+		if (!accurate)
+		{
+			nForkBlocks[i] = InvalidBlockNumber;
+			break;
+		}
+		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
+	}
+
+	for (i = 0; i < nforks; i++)
+	{
+		/*
+		* Look up the buffer in the hashtable if the block size is known to
+		* be accurate and the total blocks to be invalidated is below the
+		* full scan threshold.  Otherwise, give up the optimization.
+		*/
+		if (nForkBlocks[i] != InvalidBlockNumber &&
+			nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+		{
+			BlockNumber		curBlock;
+
+			for (curBlock = firstDelBlock[i]; curBlock < nForkBlocks[i]; curBlock++)
+			{
+				uint32		bufHash;		/* hash value for tag */
+				BufferTag	bufTag;			/* identity of requested block */
+				LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(bufTag, rnode.node, forkNum[i], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				bufHash = BufTableHashCode(&bufTag);
+				bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing. */
+				LWLockAcquire(bufPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&bufTag, bufHash);
+				LWLockRelease(bufPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[i] &&
+					bufHdr->tag.blockNum >= firstDelBlock[i])
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
+			}
+		}
+		else
+			break;
+	}
+	if (i >= nforks)
+		return;
+
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3244,8 +3328,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index a2577cb..8296aa0 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -598,7 +598,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

#115Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#114)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Mon, Oct 12, 2020 at 3:08 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Hmm. When I repeated the performance measurement for non-recovery,
I got almost similar execution results for both master and patched.

Execution Time (in seconds)
| s_b | master | patched | %reg |
|-------|--------|---------|--------|
| 128MB | 15.265 | 14.769 | -3.36% |
| 1GB | 14.808 | 14.618 | -1.30% |
| 20GB | 24.673 | 24.425 | -1.02% |
| 100GB | 74.298 | 74.813 | 0.69% |

That is considering that I removed the recovery-related checks in the patch and just
executed the commands on a standalone server.
-       if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+       if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)

Why so? Have you tried to investigate? Check if it takes an optimized
path for the non-recovery case?

--
With Regards,
Amit Kapila.

#116tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#114)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

(1)

Alright. I also removed nTotalBlocks in v24-0003 patch.

for (i = 0; i < nforks; i++)
{
if (nForkBlocks[i] != InvalidBlockNumber &&
nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
{
Optimization loop
}
else
break;
}
if (i >= nforks)
return;
{ usual buffer invalidation process }

Why do you do this way? I think the previous patch was more correct (while agreeing with Horiguchi-san in that nTotalBlocks may be unnecessary. What you want to do is "if the size of any fork could be inaccurate, do the traditional full buffer scan without performing any optimization for any fork," right? But the above code performs optimization for forks until it finds a fork with inaccurate size.

(2)
+	 * Get the total number of cached blocks and to-be-invalidated blocks
+	 * of the relation.  The cached value returned by smgrnblocks could be
+	 * smaller than the actual number of existing buffers of the file.

As you changed the meaning of the smgrnblocks() argument from cached to accurate, and you nolonger calculate the total blocks, the comment should reflect them.

(3)
In smgrnblocks(), accurate is not set to false when mdnblocks() is called. The caller doesn't initialize the value either, so it can see garbage value.

(4)
+		if (nForkBlocks[i] != InvalidBlockNumber &&
+			nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+		{
...
+		}
+		else
+			break;
+	}

In cases like this, it's better to reverse the if and else. Thus, you can reduce the nest depth.

Regards
Takayuki Tsunakawa

#117k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#116)
6 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Tuesday, October 13, 2020 10:09 AM, Tsunakawa-san wrote:

Why do you do this way? I think the previous patch was more correct (while
agreeing with Horiguchi-san in that nTotalBlocks may be unnecessary. What
you want to do is "if the size of any fork could be inaccurate, do the traditional
full buffer scan without performing any optimization for any fork," right? But
the above code performs optimization for forks until it finds a fork with
inaccurate size.

(2)
+	 * Get the total number of cached blocks and to-be-invalidated
blocks
+	 * of the relation.  The cached value returned by smgrnblocks could
be
+	 * smaller than the actual number of existing buffers of the file.

As you changed the meaning of the smgrnblocks() argument from cached to
accurate, and you nolonger calculate the total blocks, the comment should
reflect them.

(3)
In smgrnblocks(), accurate is not set to false when mdnblocks() is called.
The caller doesn't initialize the value either, so it can see garbage value.

(4)
+		if (nForkBlocks[i] != InvalidBlockNumber &&
+			nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD)
+		{
...
+		}
+		else
+			break;
+	}

In cases like this, it's better to reverse the if and else. Thus, you can reduce
the nest depth.

Thank you for the review!
1. I have revised the patch addressing your comments/feedback. Attached are the latest set of patches.

2. Non-recovery Performance
I also included a debug version of the patch (0004) where I removed the recovery-related checks
to measure non-recovery performance.
However, I still can't seem to find the cause of why the non-recovery performance
does not change when compared to master. (1 min 15 s for the given test case below)

-       if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+       if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)

Here's how I measured it:
0. postgresql.conf setting
shared_buffers = 100GB
autovacuum = off
full_page_writes = off
checkpoint_timeout = 30min
max_locks_per_transaction = 100
wal_log_hints = on
wal_keep_size = 100
max_wal_size = 20GB

1. createdb test

2. Create tables: SELECT create_tables(1000);

create or replace function create_tables(numtabs int)
returns void as $$
declare query_string text;
begin
for i in 1..numtabs loop
query_string := 'create table tab_' || i::text || ' (a int);';
execute query_string;
end loop;
end;
$$ language plpgsql;

3 Insert rows to tables (3.5 GB db): SELECT insert_tables(1000);

create or replace function insert_tables(numtabs int)
returns void as $$
declare query_string text;
begin
for i in 1..numtabs loop
query_string := 'insert into tab_' || i::text || ' SELECT generate_series(1, 100000);' ;
execute query_string;
end loop;
end;
$$ language plpgsql;

4. DELETE FROM tables: SELECT delfrom_tables(1000);

create or replace function delfrom_tables(numtabs int)
returns void as $$
declare query_string text;
begin
for i in 1..numtabs loop
query_string := 'delete from tab_' || i::text;
execute query_string;
end loop;
end;
$$ language plpgsql;

5. Measure VACUUM timing
\timing
VACUUM;

Using the debug version of the patch, I have confirmed that it enters the optimization path
when it meets the conditions. Here are some printed logs from 018_wal_optimize_node_replica.log:

make world -j4 -s && make -C src/test/recovery/ check PROVE_TESTS=t/018_wal_optimize.pl

WARNING: current fork 0, nForkBlocks[i] 1, accurate: 1
CONTEXT: WAL redo at 0/162B4E0 for Storage/TRUNCATE: base/13751/24577 to 0 blocks flags 7
WARNING: Optimization Loop.
buf_id = 41. nforks = 1. current fork = 0. forkNum: 0 == tag's forkNum: 0. curBlock: 0 < nForkBlocks[i] = 1. tag blockNum: 0 >= firstDelBlock[i]: 0. nBlocksToInvalidate = 1 < threshold = 32.

--
3. Recovery Performance (hot standby, failover)
OTOH, when executing recovery performance (using 0003 patch), the results were great.

| s_b | master | patched | %reg |
|-------|--------|---------|--------|
| 128MB | 3.043 | 2.977 | -2.22% |
| 1GB | 3.417 | 3.41 | -0.21% |
| 20GB | 20.597 | 2.409 | -755% |
| 100GB | 66.862 | 2.409 | -2676% |

To execute this on a hot standby setup (after inserting rows to tables)
1. [Standby] Pause WAL replay
SELECT pg_wal_replay_pause();

2. [Master] Measure VACUUM timing. Then stop server.
\timing
VACUUM;
\q
pg_ctl stop -mi -w

3. [Standby] Use the attached script to promote standby and measure the performance.
# test.sh recovery

So the current issue I'm still investigating is why the performance for non-recovery is bad,
while OTOH it's good when InRecovery.

Regards,
Kirk Jamison

Attachments:

v25-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v25-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From 815071a039fd7a865eeccb8b5fb62467bac2d126 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v25 1/4] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v25-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v25-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From 83c9b3227673fe93a389c0043d42947aa219c039 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v25 2/4] Add bool param in smgrnblocks() for cached blocks.

---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 12 ++++++++++--
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 26 insertions(+), 17 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 7e915bc..46fd97d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index dbbd3aa..4dc2ba8 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -720,7 +720,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..ae001fe 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -739,7 +739,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2855,7 +2855,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..a2577cb 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -553,16 +553,24 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
  *					 supplied relation.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  The cached values
+	 * could be smaller than the actual number of existing buffers of the file.
+	 * This is caused by lseek of buggy Linux kernels that might not have
+	 * accounted for the recent write.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (accurate != NULL)
+			*accurate = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..af11b53 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *accurate);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v25-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v25-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 41a93fa4b09510b01fef722568428f7332ae082b Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 9 Oct 2020 08:59:55 +0000
Subject: [PATCH v25 3/4] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 100 ++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     |   5 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 96 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ae001fe..5e52ff0 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2965,18 +2967,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are known to be accurate. See smgrnblocks() for
+ *		more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		accurate;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2990,10 +3002,81 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of to-be-invalidated blocks of a relation as well
+	 * as the total nblocks for a given fork. The cached value returned by
+	 * smgrnblocks could be smaller than the actual number of existing buffers
+	 * of the file.  This is caused by buggy Linux kernels that might not have
+	 * accounted for the recent write.  Give up the optimization if the block
+	 * count of any fork cannot be trusted.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the total nblocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &accurate);
+
+		if (!accurate)
+		{
+			nForkBlocks[i] = InvalidBlockNumber;
+			break;
+		}
+
+		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffer in the hashtable if the block size is known to
+	 * be accurate and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		BlockNumber		curBlock;
+
+		if (nForkBlocks[i] == InvalidBlockNumber ||
+			nBlocksToInvalidate >= BUF_DROP_FULL_SCAN_THRESHOLD)
+			break;
+
+		for (curBlock = firstDelBlock[i]; curBlock < nForkBlocks[i]; curBlock++)
+		{
+			uint32		bufHash;		/* hash value for tag */
+			BufferTag	bufTag;			/* identity of requested block */
+			LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+			int		buf_id;
+
+			/* create a tag so we can lookup the buffer */
+			INIT_BUFFERTAG(bufTag, rnode.node, forkNum[i], curBlock);
+
+			/* determine its hash code and partition lock ID */
+			bufHash = BufTableHashCode(&bufTag);
+			bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+			/* Check that it is in the buffer pool. If not, do nothing. */
+			LWLockAcquire(bufPartitionLock, LW_SHARED);
+			buf_id = BufTableLookup(&bufTag, bufHash);
+			LWLockRelease(bufPartitionLock);
+
+			if (buf_id < 0)
+				continue;
+
+			bufHdr = GetBufferDescriptor(buf_id);
+
+			buf_state = LockBufHdr(bufHdr);
+
+			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+				bufHdr->tag.forkNum == forkNum[i] &&
+				bufHdr->tag.blockNum >= firstDelBlock[i])
+				InvalidateBuffer(bufHdr);	/* releases spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);
+		}
+	}
+	if (i >= nforks)
+		return;
+
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3244,8 +3327,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index a2577cb..6b8528e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -572,6 +572,9 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
 		return reln->smgr_cached_nblocks[forknum];
 	}
 
+	if (accurate != NULL)
+		*accurate = false;
+
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
 	reln->smgr_cached_nblocks[forknum] = result;
@@ -598,7 +601,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v25-0004-V25-with-ereport-for-debug.patchapplication/octet-stream; name=v25-0004-V25-with-ereport-for-debug.patchDownload
From a03db42d1e75a5ee587e6cf31cd11592024eed5d Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Thu, 15 Oct 2020 02:55:59 +0000
Subject: [PATCH v25 4/4] V25 with ereport for debug

---
 src/backend/storage/buffer/bufmgr.c | 22 ++++++++++++++++++++++
 src/backend/storage/smgr/smgr.c     |  2 +-
 2 files changed, 23 insertions(+), 1 deletion(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 5e52ff0..b1aaa05 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3015,6 +3015,10 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 		/* Get the total nblocks for a relation's fork */
 		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &accurate);
 
+		ereport(WARNING,
+				(errmsg("current fork %i, nForkBlocks[i] %u, accurate: %d",
+						 i, nForkBlocks[i], accurate)));
+
 		if (!accurate)
 		{
 			nForkBlocks[i] = InvalidBlockNumber;
@@ -3066,7 +3070,19 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
 				bufHdr->tag.forkNum == forkNum[i] &&
 				bufHdr->tag.blockNum >= firstDelBlock[i])
+			{
+				ereport(WARNING,
+						(errmsg("Optimization Loop.\n"
+								"buf_id = %i. nforks = %i. current fork = %u. "
+								"forkNum: %u == tag's forkNum: %u. "
+								"curBlock: %u  <  nForkBlocks[i] = %u. "
+								"tag blockNum: %u  >=  firstDelBlock[i]: %u. "
+								"nBlocksToInvalidate = %u < threshold = %u. ",
+								buf_id, nforks, i, forkNum[i], bufHdr->tag.forkNum,
+								curBlock, nForkBlocks[i], bufHdr->tag.blockNum, firstDelBlock[i],
+								nBlocksToInvalidate, BUF_DROP_FULL_SCAN_THRESHOLD)));
 				InvalidateBuffer(bufHdr);	/* releases spinlock */
+			}
 			else
 				UnlockBufHdr(bufHdr, buf_state);
 		}
@@ -3105,6 +3121,12 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 				bufHdr->tag.forkNum == forkNum[j] &&
 				bufHdr->tag.blockNum >= firstDelBlock[j])
 			{
+				ereport(WARNING,
+						(errmsg("Full Scan.\n"
+								"nforks = %i.   tag's forkNum %u == forkNum %u.  "
+								"tag blockNum %u  >= firstDelBlock[j] %u.",
+								nforks, bufHdr->tag.forkNum, forkNum[j],
+								bufHdr->tag.blockNum, firstDelBlock[j])));
 				InvalidateBuffer(bufHdr);	/* releases spinlock */
 				break;
 			}
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 6b8528e..f57916f 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -564,7 +564,7 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
 	 * This is caused by lseek of buggy Linux kernels that might not have
 	 * accounted for the recent write.
 	 */
-	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
 	{
 		if (accurate != NULL)
 			*accurate = true;
-- 
1.8.3.1

018_wal_optimize_node_replica.logapplication/octet-stream; name=018_wal_optimize_node_replica.logDownload
test.shapplication/octet-stream; name=test.shDownload
#118tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#117)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

2. Non-recovery Performance
However, I still can't seem to find the cause of why the non-recovery
performance does not change when compared to master. (1 min 15 s for the
given test case below)

...

5. Measure VACUUM timing
\timing
VACUUM;

Oops, why are you using VACUUM? Aren't you trying to speed up TRUNCATE?

Even if you wanted to utilize the truncation at the end of VACUUM for measuring truncation speed, your way measures the whole VACUUM processing, which includes the garbage collection process. The garbage collection should dominate the time.

3. Recovery Performance (hot standby, failover) OTOH, when executing
2. [Master] Measure VACUUM timing. Then stop server.
\timing
VACUUM;
\q
pg_ctl stop -mi -w

3. [Standby] Use the attached script to promote standby and measure the
performance.
# test.sh recovery

You didn't DELETE the table data as opposed to the non-recovery case. Then, the replay of VACUUM should do nothing. That's why you got a good performance number.

TRUNCATE goes this path:

[non-recovery]
CommitTransaction
smgrdopendingdeletes
smgrdounlinkall
DropRelFileNodesAllBuffers

[recovery]
xact_redo_commit
DropRelationFiles
smgrdounlinkall
DropRelFileNodesAllBuffers

So, you need to modify DropRelFileNodesAllBuffers(). OTOH, DropRelFileNodeBuffers(), which you modified, is used in VACUUM's truncation and another case. The modification itself is useful because it can shorten the occasional hickup during autovacuum, so you don't remove the change.

(The existence of these two paths is tricky; anyone on this thread didn't notice, and I forgot about it. It would be good to refactor this, but it's a separate undertaking, I think.)

Below are my comments for the code:

(1)
@@ -572,6 +572,9 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
+	if (accurate != NULL)
+		*accurate = false;
+

The above change should be in 002, right?

(2)
+ /* Get the total nblocks for a relation's fork */

total nblocks -> number of blocks

(3)
+		if (nForkBlocks[i] == InvalidBlockNumber ||
+			nBlocksToInvalidate >= BUF_DROP_FULL_SCAN_THRESHOLD)
+			break;

With this code, you haven't addressed what I commented previously. If the size of the first fork is accurate but that of the second one is not, the first fork is processed in an optimized way while the second fork is done in the traditional way. What you want to here is to only use the traditional way for all forks, right?

So, remove the above change and replace

+		if (!accurate)
+		{
+			nForkBlocks[i] = InvalidBlockNumber;
+			break;
+		}

with

+ if (!accurate)
+ break;

And after the first for loop, put

if (!accurate || nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
goto full_scan;

And remove the following code and instead put the "full_scan:" label there.

+	if (i >= nforks)
+		return;
+

Or, instead of using goto, you can write like this:

for (...)
calculate # of invalidated blocks

if (accurate && nBlocksToInvalidate >= BUF_DROP_FULL_SCAN_THRESHOLD)
{
do the optimized way;
return;
}

do the traditional way;

I prefer using goto here because the loop nesting gets shallow. But that's a matter of taste and you can choose either.

Regards
Takayuki Tsunakawa

#119tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#117)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

However, I still can't seem to find the cause of why the non-recovery
performance does not change when compared to master. (1 min 15 s for the
given test case below)

Can you check and/or try the following?

1. Isn't the vacuum cost delay working?
VACUUM command should run without sleeping with the default settings. Just in case, can you try with the settings:

vacuum_cost_delay = 0
vacuum_cost_limit = 10000

2. Buffer strategy
The non-recovery VACUUM can differ from that of recovery in the use of shared buffers. The VACUUM command uses only 256 KB of shared buffers. To make VACUUM command use the whole shared buffers, can you modify src/backend/commands/vacuum.c so that GetAccessStrategy()'s argument is changed to BAS_VACUUM to BAS_NORMAL? (I don't have much hope about this, though, because all blocks of the relations are already cached in shared buffers when VACUUM is run.)

Can you measure the time DropRelFileNodeBuffers()? You can call GetTimestamp() at the beginning and end of the function, and use TimestampDifference() to calculate the difference. Then, for instance, elog(WARNING, "time is | %u.%u", sec, usec) at the end of the function. You can use any elog() print format for your convenience to write shell commands to filter the lines and sum up the total.

Regards
Takayuki Tsunakawa

#120tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#119)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: tsunakawa.takay@fujitsu.com <tsunakawa.takay@fujitsu.com>

Can you measure the time DropRelFileNodeBuffers()? You can call
GetTimestamp() at the beginning and end of the function, and use
TimestampDifference() to calculate the difference. Then, for instance,
elog(WARNING, "time is | %u.%u", sec, usec) at the end of the function. You
can use any elog() print format for your convenience to write shell commands to
filter the lines and sum up the total.

Before doing this, you can also do "VACUUM (truncate off)" to see which of the garbage collection or relation truncation takes long time. The relation truncation processing includes not only DropRelFileNodeBuffers() but also file truncation and something else, but it's an easy filter.

Regards
Takayuki Tsunakawa

#121tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#120)
RE: [Patch] Optimize dropping of relation buffers using dlist

RelationTruncate() invalidates the cached fork sizes as follows. This causes smgrnblocks() return accurate=false, resulting in not running optimization. Try commenting out for non-recovery case.

/*
* Make sure smgr_targblock etc aren't pointing somewhere past new end
*/
rel->rd_smgr->smgr_targblock = InvalidBlockNumber;
for (int i = 0; i <= MAX_FORKNUM; ++i)
rel->rd_smgr->smgr_cached_nblocks[i] = InvalidBlockNumber;

Regards
Takayuki Tsunakawa

#122k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#121)
4 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wednesday, October 21, 2020 4:37 PM, Tsunakawa-san wrote:

RelationTruncate() invalidates the cached fork sizes as follows. This causes
smgrnblocks() return accurate=false, resulting in not running optimization.
Try commenting out for non-recovery case.

/*
* Make sure smgr_targblock etc aren't pointing somewhere past new
end
*/
rel->rd_smgr->smgr_targblock = InvalidBlockNumber;
for (int i = 0; i <= MAX_FORKNUM; ++i)
rel->rd_smgr->smgr_cached_nblocks[i] = InvalidBlockNumber;

Hello, I have updated the set of patches which incorporated all your feedback in the previous email.
Thank you for also looking into it. The patch 0003 (DropRelFileNodeBuffers improvement)
is indeed for vacuum optimization and not for truncate.
I'll post a separate patch for the truncate optimization in the coming days.

1. Vacuum Optimization
I have confirmed that the above comment (commenting out the lines in RelationTruncate)
solves the issue for non-recovery case.
The attached 0004 patch is just for non-recovery testing and is not included in the
final set of patches to be committed for vacuum optimization.

The table below shows the vacuum execution time for non-recovery case.
I've also subtracted the execution time when VACUUM (truncate off) is set.

[NON-RECOVERY CASE - VACUUM execution Time in seconds]

| s_b | master | patched | %reg |
|-------|--------|---------|-----------|
| 128MB | 0.22 | 0.181 | -21.55% |
| 1GB | 0.701 | 0.712 | 1.54% |
| 20GB | 15.027 | 1.920 | -682.66% |
| 100GB | 65.456 | 1.795 | -3546.57% |

[RECOVERY CASE, VACUUM execution + failover]
I've made a mistake in my writing of the previous email [1]/messages/by-id/OSBPR01MB2341672E9A95E5EC6D2E79B5EF020@OSBPR01MB2341.jpnprd01.prod.outlook.com.
DELETE from was executed before pausing the WAL replay on standby.
In short, the procedure and results were correct. But I repeated the
performance measurement just in case. The results are still great and
almost the same as the previous measurement.

| s_b | master | patched | %reg |
|-------|--------|---------|--------|
| 128MB | 3.043 | 3.009 | -1.13% |
| 1GB | 3.417 | 3.410 | -0.21% |
| 20GB | 20.597 | 2.410 | -755% |
| 100GB | 65.734 | 2.409 | -2629% |

Based from the results above, with the patches applied,
the performance for both recovery and non-recovery were relatively close.
For default and small shared_buffers (128MB, 1GB), the performance is
relatively the same as master. But we see the benefit when we have large shared_buffers setting.

I've tested using the same test case I indicated in the previous email,
Including the following additional setting:
vacuum_cost_delay = 0
vacuum_cost_limit = 10000

That's it for the vacuum optimization. Feedback and comments would be highly appreciated.

2. Truncate Optimization
I'll post a separate patch in the future for the truncate optimization which modifies the
DropRelFileNodesAllBuffers and related functions along the truncate path..

Thank you.

Regards,
Kirk Jamison

[1]: /messages/by-id/OSBPR01MB2341672E9A95E5EC6D2E79B5EF020@OSBPR01MB2341.jpnprd01.prod.outlook.com

Attachments:

v26-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v26-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From a700c30472b73bbf65fb2d7cc2be2a1ac351ae5b Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v26 1/4] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v26-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v26-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From f64f688f39343b9e11115d7c42085d2f85346c9b Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v26 2/4] Add bool param in smgrnblocks() for cached blocks.

---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 15 +++++++++++++--
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 29 insertions(+), 17 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 7e915bc..46fd97d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index dbbd3aa..4dc2ba8 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -720,7 +720,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..ae001fe 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -739,7 +739,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2855,7 +2855,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..aad6e5d 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -553,16 +553,27 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
  *					 supplied relation.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  The cached values
+	 * could be smaller than the actual number of existing buffers of the file.
+	 * This is caused by lseek of buggy Linux kernels that might not have
+	 * accounted for the recent write.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (accurate != NULL)
+			*accurate = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
+
+	if (accurate != NULL)
+		*accurate = false;
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..af11b53 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *accurate);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v26-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v26-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 50f78225366c1ea34e81e6441baa22109b7cb5d1 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 16 Oct 2020 02:23:05 +0000
Subject: [PATCH v26 3/4] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 94 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     |  2 +-
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 87 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ae001fe..fb6ba94 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2965,18 +2967,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are known to be accurate. See smgrnblocks() for
+ *		more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		accurate;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2990,10 +3002,75 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of to-be-invalidated blocks of a relation as well
+	 * as the total nblocks for a given fork. The cached value returned by
+	 * smgrnblocks could be smaller than the actual number of existing buffers
+	 * of the file.  This is caused by buggy Linux kernels that might not have
+	 * accounted for the recent write.  Give up the optimization if the block
+	 * count of any fork cannot be trusted.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the total nblocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &accurate);
+
+		if (!accurate)
+			break;
+
+		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffer in the hashtable if the block size is known to
+	 * be accurate and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (accurate && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		curBlock;
+
+			for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j]; curBlock++)
+			{
+				uint32		bufHash;		/* hash value for tag */
+				BufferTag	bufTag;			/* identity of requested block */
+				LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				bufHash = BufTableHashCode(&bufTag);
+				bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing. */
+				LWLockAcquire(bufPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&bufTag, bufHash);
+				LWLockRelease(bufPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= firstDelBlock[j])
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3244,8 +3321,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index aad6e5d..6b8528e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -601,7 +601,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v26-0004-For-non-recovery-performance-test-case-purposes-.patchapplication/octet-stream; name=v26-0004-For-non-recovery-performance-test-case-purposes-.patchDownload
From c6746d21ae209382170700f8fe1f61ef063859d7 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Mon, 19 Oct 2020 07:34:39 +0000
Subject: [PATCH v26 4/4] For non-recovery performance test case purposes only.

---
 src/backend/catalog/storage.c       | 6 +++---
 src/backend/commands/vacuum.c       | 2 +-
 src/backend/storage/buffer/bufmgr.c | 5 +++--
 src/backend/storage/smgr/smgr.c     | 2 +-
 4 files changed, 8 insertions(+), 7 deletions(-)

diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index 4dc2ba8..32d7ba8 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -289,9 +289,9 @@ RelationTruncate(Relation rel, BlockNumber nblocks)
 	/*
 	 * Make sure smgr_targblock etc aren't pointing somewhere past new end
 	 */
-	rel->rd_smgr->smgr_targblock = InvalidBlockNumber;
-	for (int i = 0; i <= MAX_FORKNUM; ++i)
-		rel->rd_smgr->smgr_cached_nblocks[i] = InvalidBlockNumber;
+	//rel->rd_smgr->smgr_targblock = InvalidBlockNumber;
+	//for (int i = 0; i <= MAX_FORKNUM; ++i)
+	//	rel->rd_smgr->smgr_cached_nblocks[i] = InvalidBlockNumber;
 
 	/* Prepare for truncation of MAIN fork of the relation */
 	forks[nforks] = MAIN_FORKNUM;
diff --git a/src/backend/commands/vacuum.c b/src/backend/commands/vacuum.c
index ddeec87..b6162f0 100644
--- a/src/backend/commands/vacuum.c
+++ b/src/backend/commands/vacuum.c
@@ -343,7 +343,7 @@ vacuum(List *relations, VacuumParams *params,
 	{
 		MemoryContext old_context = MemoryContextSwitchTo(vac_context);
 
-		bstrategy = GetAccessStrategy(BAS_VACUUM);
+		bstrategy = GetAccessStrategy(BAS_NORMAL);
 		MemoryContextSwitchTo(old_context);
 	}
 	vac_strategy = bstrategy;
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index fb6ba94..15c14f6 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3004,7 +3004,7 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 
 	/*
 	 * Get the total number of to-be-invalidated blocks of a relation as well
-	 * as the total nblocks for a given fork. The cached value returned by
+	 * as the total nblocks for a given fork.  The cached value returned by
 	 * smgrnblocks could be smaller than the actual number of existing buffers
 	 * of the file.  This is caused by buggy Linux kernels that might not have
 	 * accounted for the recent write.  Give up the optimization if the block
@@ -3012,12 +3012,13 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 	 */
 	for (i = 0; i < nforks; i++)
 	{
-		/* Get the total nblocks for a relation's fork */
+		/* Get the number of blocks for a relation's fork */
 		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &accurate);
 
 		if (!accurate)
 			break;
 
+		/* Get the number of blocks to be invalidated */
 		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
 	}
 
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 6b8528e..f57916f 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -564,7 +564,7 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
 	 * This is caused by lseek of buggy Linux kernels that might not have
 	 * accounted for the recent write.
 	 */
-	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
 	{
 		if (accurate != NULL)
 			*accurate = true;
-- 
1.8.3.1

#123tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#122)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

I have confirmed that the above comment (commenting out the lines in
RelationTruncate) solves the issue for non-recovery case.
The attached 0004 patch is just for non-recovery testing and is not included in
the final set of patches to be committed for vacuum optimization.

I'm relieved to hear that.

As for 0004:
When testing TRUNCATE, remove the change to storage.c because it was intended to troubleshoot the VACUUM test.
What's the change in bufmgr.c for? Is it to be included in 0001 or 0002?

The table below shows the vacuum execution time for non-recovery case.
I've also subtracted the execution time when VACUUM (truncate off) is set.

[NON-RECOVERY CASE - VACUUM execution Time in seconds]

(snip)

| 100GB | 65.456 | 1.795 | -3546.57% |

So, the full shared buffer scan for 10,000 relations took about as long as 63 seconds (= 6.3 ms per relation). It's nice to shorten this long time.

I'll review the patch soon.

Regards
Takayuki Tsunakawa

#124tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#123)
RE: [Patch] Optimize dropping of relation buffers using dlist

As for 0004:
When testing TRUNCATE, remove the change to storage.c because it was
intended to troubleshoot the VACUUM test.

I meant vacuum.c. Sorry.

Regards
Takayuki Tsunakawa

#125tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#122)
RE: [Patch] Optimize dropping of relation buffers using dlist

The patch looks good except for the minor one:

(1)
+ * as the total nblocks for a given fork. The cached value returned by

nblocks -> blocks

Regards
Takayuki Tsunakawa

#126k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#123)
4 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, October 22, 2020 10:34 AM, Tsunakwa-san wrote:

I have confirmed that the above comment (commenting out the lines in
RelationTruncate) solves the issue for non-recovery case.
The attached 0004 patch is just for non-recovery testing and is not
included in the final set of patches to be committed for vacuum

optimization.

I'm relieved to hear that.

As for 0004:
When testing TRUNCATE, remove the change to storage.c because it was
intended to troubleshoot the VACUUM test.

I've removed it now.

What's the change in bufmgr.c for? Is it to be included in 0001 or 0002?

Right. But that should be in 0003. Fixed.

I also fixed the feedback from the previous email:

(1)
+ * as the total nblocks for a given fork. The cached value returned by

nblocks -> blocks

The table below shows the vacuum execution time for non-recovery case.
I've also subtracted the execution time when VACUUM (truncate off) is set.

[NON-RECOVERY CASE - VACUUM execution Time in seconds]

(snip)

| 100GB | 65.456 | 1.795 | -3546.57% |

So, the full shared buffer scan for 10,000 relations took about as long as 63
seconds (= 6.3 ms per relation). It's nice to shorten this long time.

I'll review the patch soon.

Thank you very much for the reviews. Attached are the latest set of patches.

Regards,
Kirk Jamison

Attachments:

v27-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v27-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From 55238405df708a1bad6b2df951298e2e60cf50bd Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v27 1/4] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v27-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v27-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From 0c4fda9460a21d97d0b45f88fa62936fa3c9d517 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v27 2/4] Add bool param in smgrnblocks() for cached blocks.

---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 15 +++++++++++++--
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 29 insertions(+), 17 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 7e915bc..46fd97d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index dbbd3aa..4dc2ba8 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -720,7 +720,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e549fa1..ae001fe 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -739,7 +739,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2855,7 +2855,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..aad6e5d 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -553,16 +553,27 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
  *					 supplied relation.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  The cached values
+	 * could be smaller than the actual number of existing buffers of the file.
+	 * This is caused by lseek of buggy Linux kernels that might not have
+	 * accounted for the recent write.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (accurate != NULL)
+			*accurate = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
+
+	if (accurate != NULL)
+		*accurate = false;
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..af11b53 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *accurate);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v27-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v27-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 8cc8e8dad064ae1e467f1281e71028396ccac9b6 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 16 Oct 2020 02:23:05 +0000
Subject: [PATCH v27 3/4] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning. This improves the
performance especially when VACUUM or autovacuum truncated off any
of the empty pages at the end of relation.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 95 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     |  2 +-
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 88 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ae001fe..20c806e 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2965,18 +2967,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are known to be accurate. See smgrnblocks() for
+ *		more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		accurate;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2990,10 +3002,76 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of to-be-invalidated blocks of a relation as well
+	 * as the total blocks for a given fork.  The cached value returned by
+	 * smgrnblocks could be smaller than the actual number of existing buffers
+	 * of the file.  This is caused by buggy Linux kernels that might not have
+	 * accounted for the recent write.  Give up the optimization if the block
+	 * count of any fork cannot be trusted.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &accurate);
+
+		if (!accurate)
+			break;
+
+		/* Get the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffer in the hashtable if the block size is known to
+	 * be accurate and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (accurate && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		curBlock;
+
+			for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j]; curBlock++)
+			{
+				uint32		bufHash;		/* hash value for tag */
+				BufferTag	bufTag;			/* identity of requested block */
+				LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				bufHash = BufTableHashCode(&bufTag);
+				bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing. */
+				LWLockAcquire(bufPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&bufTag, bufHash);
+				LWLockRelease(bufPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= firstDelBlock[j])
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3244,8 +3322,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index aad6e5d..6b8528e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -601,7 +601,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v27-0004-For-non-recovery-performance-test-case-purposes-.patchapplication/octet-stream; name=v27-0004-For-non-recovery-performance-test-case-purposes-.patchDownload
From dd6af1d00268ace32ac3390596c16e9c1688a5f4 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Thu, 22 Oct 2020 01:57:49 +0000
Subject: [PATCH v27 4/4] For non-recovery performance test case purposes only.

Not included in the set of patches to be committed.
---
 src/backend/catalog/storage.c   | 6 +++---
 src/backend/storage/smgr/smgr.c | 2 +-
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index 4dc2ba8..32d7ba8 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -289,9 +289,9 @@ RelationTruncate(Relation rel, BlockNumber nblocks)
 	/*
 	 * Make sure smgr_targblock etc aren't pointing somewhere past new end
 	 */
-	rel->rd_smgr->smgr_targblock = InvalidBlockNumber;
-	for (int i = 0; i <= MAX_FORKNUM; ++i)
-		rel->rd_smgr->smgr_cached_nblocks[i] = InvalidBlockNumber;
+	//rel->rd_smgr->smgr_targblock = InvalidBlockNumber;
+	//for (int i = 0; i <= MAX_FORKNUM; ++i)
+	//	rel->rd_smgr->smgr_cached_nblocks[i] = InvalidBlockNumber;
 
 	/* Prepare for truncation of MAIN fork of the relation */
 	forks[nforks] = MAIN_FORKNUM;
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 6b8528e..f57916f 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -564,7 +564,7 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
 	 * This is caused by lseek of buggy Linux kernels that might not have
 	 * accounted for the recent write.
 	 */
-	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	if (reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
 	{
 		if (accurate != NULL)
 			*accurate = true;
-- 
1.8.3.1

#127Thomas Munro
thomas.munro@gmail.com
In reply to: k.jamison@fujitsu.com (#126)
Re: [Patch] Optimize dropping of relation buffers using dlist
On Thu, Oct 22, 2020 at 3:07 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:
+    /*
+     * Get the total number of to-be-invalidated blocks of a relation as well
+     * as the total blocks for a given fork.  The cached value returned by
+     * smgrnblocks could be smaller than the actual number of existing buffers
+     * of the file.  This is caused by buggy Linux kernels that might not have
+     * accounted for the recent write.  Give up the optimization if the block
+     * count of any fork cannot be trusted.
+     */
+    for (i = 0; i < nforks; i++)
+    {
+        /* Get the number of blocks for a relation's fork */
+        nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &accurate);
+
+        if (!accurate)
+            break;

Hmmm. The Linux comment led me to commit ffae5cc and a 2006 thread[1]/messages/by-id/26202.1159032931@sss.pgh.pa.us
showing a buggy sequence of system calls. AFAICS it was not even an
SMP/race problem of the type you might half expect, it was a single
process not seeing its own write. I didn't find details on the
version, filesystem etc.

Searching for our message "This has been seen to occur with buggy
kernels; consider updating your system" turns up recent-ish results
too. The reports I read involved GlusterFS, which I don't personally
know anything about, but it claims full POSIX compliance, and POSIX is
strict about that sort of thing, so I'd guess that is/was a fairly
serious bug or misconfiguration. Surely there must be other symptoms
for PostgreSQL on such systems too, like sequential scans that don't
see recently added pages.

But... does the proposed caching behaviour and "accurate" flag really
help with any of that? Cached values come from lseek() anyway. If we
just trusted unmodified smgrnblocks(), someone running on such a
forgetful file system might eventually see nasty errors because we
left buffers in the buffer pool that prevent a checkpoint from
completing (and panic?), but they might also see other really strange
errors, and that applies with or without that "accurate" flag, no?

[1]: /messages/by-id/26202.1159032931@sss.pgh.pa.us

#128Tom Lane
tgl@sss.pgh.pa.us
In reply to: Thomas Munro (#127)
Re: [Patch] Optimize dropping of relation buffers using dlist

Thomas Munro <thomas.munro@gmail.com> writes:

Hmmm. The Linux comment led me to commit ffae5cc and a 2006 thread[1]
showing a buggy sequence of system calls.

Hah, blast from the past ...

AFAICS it was not even an
SMP/race problem of the type you might half expect, it was a single
process not seeing its own write. I didn't find details on the
version, filesystem etc.

Per the referenced bug-reporting thread, it was ReiserFS and/or NFS on
SLES 9.3; so, dubious storage choices on an ancient-even-then Linux
kernel.

I think the takeaway point is not so much that that particular bug
might recur as that storage infrastructure does sometimes have bugs.
If you're wanting to introduce new assumptions about what the filesystem
will do, it's prudent to think about how badly will we break if the
assumptions fail.

regards, tom lane

#129Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Thomas Munro (#127)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 22 Oct 2020 16:35:27 +1300, Thomas Munro <thomas.munro@gmail.com> wrote in

On Thu, Oct 22, 2020 at 3:07 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:
+    /*
+     * Get the total number of to-be-invalidated blocks of a relation as well
+     * as the total blocks for a given fork.  The cached value returned by
+     * smgrnblocks could be smaller than the actual number of existing buffers
+     * of the file.  This is caused by buggy Linux kernels that might not have
+     * accounted for the recent write.  Give up the optimization if the block
+     * count of any fork cannot be trusted.
+     */
+    for (i = 0; i < nforks; i++)
+    {
+        /* Get the number of blocks for a relation's fork */
+        nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &accurate);
+
+        if (!accurate)
+            break;

Hmmm. The Linux comment led me to commit ffae5cc and a 2006 thread[1]
showing a buggy sequence of system calls. AFAICS it was not even an
SMP/race problem of the type you might half expect, it was a single
process not seeing its own write. I didn't find details on the
version, filesystem etc.

Anyway that comment is irrelevant to the added code. The point here is
that the returned value may not be reliable, due to not only the
kernel bugs, but the files is extended/truncated by other
procesess. But I suppose that we may have synchronized file-size cache
in the future?

Searching for our message "This has been seen to occur with buggy
kernels; consider updating your system" turns up recent-ish results
too. The reports I read involved GlusterFS, which I don't personally
know anything about, but it claims full POSIX compliance, and POSIX is
strict about that sort of thing, so I'd guess that is/was a fairly
serious bug or misconfiguration. Surely there must be other symptoms
for PostgreSQL on such systems too, like sequential scans that don't
see recently added pages.

But... does the proposed caching behaviour and "accurate" flag really
help with any of that? Cached values come from lseek() anyway. If we

That "accurate" (good name wanted) flag suggest that it is guaranteed
that we don't have a buffer for blocks after that block number.

just trusted unmodified smgrnblocks(), someone running on such a
forgetful file system might eventually see nasty errors because we
left buffers in the buffer pool that prevent a checkpoint from
completing (and panic?), but they might also see other really strange
errors, and that applies with or without that "accurate" flag, no?

[1] /messages/by-id/26202.1159032931@sss.pgh.pa.us

smgrtruncate and msgrextend modifies that cache from their parameter,
not from lseek(). At the very first the value in the cache comes from
lseek() but if nothing other than postgres have changed the file size,
I believe we can rely on the cache even with such a buggy kernels even
if still exists.

If there's no longer such a buggy kernel, we can rely on lseek() only
when InRecovery. If we had synchronized file size cache we could rely
on the cache even while !InRecovery. (I'm not sure about how vacuum
affects, though.)

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#130Thomas Munro
thomas.munro@gmail.com
In reply to: Tom Lane (#128)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Oct 22, 2020 at 5:52 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Per the referenced bug-reporting thread, it was ReiserFS and/or NFS on
SLES 9.3; so, dubious storage choices on an ancient-even-then Linux
kernel.

Ohhhh. I can reproduce that on a modern Linux box by forcing
writeback to a full NFS filesystem[1]/messages/by-id/CAEepm=1FGo=ACPKRmAxvb53mBwyVC=TDwTE0DMzkWjdbAYw7sw@mail.gmail.com, approximately as the kernel
does asynchronously when it feels like it, causing the size reported
by SEEK_END to go down.

$ cat magic_shrinking_file.c
#include <fcntl.h>
#include <stdio.h>
#include <stdlib.h>
#include <unistd.h>

int main()
{
int fd;
char buffer[8192] = {0};

fd = open("/mnt/test_loopback_remote/dir/file", O_RDWR | O_APPEND);
if (fd < 0) {
perror("open");
return EXIT_FAILURE;
}
printf("lseek(..., SEEK_END) = %jd\n", lseek(fd, 0, SEEK_END));
printf("write(...) = %zd\n", write(fd, buffer, sizeof(buffer)));
printf("lseek(..., SEEK_END) = %jd\n", lseek(fd, 0, SEEK_END));
printf("fsync(...) = %d\n", fsync(fd));
printf("lseek(..., SEEK_END) = %jd\n", lseek(fd, 0, SEEK_END));

return EXIT_SUCCESS;
}
$ cc magic_shrinking_file.c
$ ./a.out
lseek(..., SEEK_END) = 9670656
write(...) = 8192
lseek(..., SEEK_END) = 9678848
fsync(...) = -1
lseek(..., SEEK_END) = 9670656

I think the takeaway point is not so much that that particular bug
might recur as that storage infrastructure does sometimes have bugs.
If you're wanting to introduce new assumptions about what the filesystem
will do, it's prudent to think about how badly will we break if the
assumptions fail.

Yeah. My point was just that the caching trick doesn't seem to
improve matters on this particular front, it can just cache a bogus
value.

[1]: /messages/by-id/CAEepm=1FGo=ACPKRmAxvb53mBwyVC=TDwTE0DMzkWjdbAYw7sw@mail.gmail.com

#131Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#123)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 22 Oct 2020 01:33:31 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

The table below shows the vacuum execution time for non-recovery case.
I've also subtracted the execution time when VACUUM (truncate off) is set.

[NON-RECOVERY CASE - VACUUM execution Time in seconds]

(snip)

| 100GB | 65.456 | 1.795 | -3546.57% |

So, the full shared buffer scan for 10,000 relations took about as long as 63 seconds (= 6.3 ms per relation). It's nice to shorten this long time.

I'm not sure about the exact steps of the test, but it can be expected
if we have many small relations to truncate.

Currently BUF_DROP_FULL_SCAN_THRESHOLD is set to Nbuffers / 512, which
is quite arbitrary that comes from a wild guess.

Perhaps we need to run benchmarks that drops one relation with several
different ratios between the number of buffers to-be-dropped and
Nbuffers, and preferably both on spinning rust and SSD.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#132Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Kyotaro Horiguchi (#129)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 22 Oct 2020 14:16:37 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

At Thu, 22 Oct 2020 16:35:27 +1300, Thomas Munro <thomas.munro@gmail.com> wrote in

On Thu, Oct 22, 2020 at 3:07 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:
But... does the proposed caching behaviour and "accurate" flag really
help with any of that? Cached values come from lseek() anyway. If we

That "accurate" (good name wanted) flag suggest that it is guaranteed
that we don't have a buffer for blocks after that block number.

just trusted unmodified smgrnblocks(), someone running on such a
forgetful file system might eventually see nasty errors because we
left buffers in the buffer pool that prevent a checkpoint from
completing (and panic?), but they might also see other really strange
errors, and that applies with or without that "accurate" flag, no?

[1] /messages/by-id/26202.1159032931@sss.pgh.pa.us

smgrtruncate and msgrextend modifies that cache from their parameter,
not from lseek(). At the very first the value in the cache comes from
lseek() but if nothing other than postgres have changed the file size,
I believe we can rely on the cache even with such a buggy kernels even
if still exists.

Mmm. Not exact. The requirement here is that we must be certain that
the we don't have a buffuer for blocks after the file size known to
the process. While recoverying, If the first lseek() returned smaller
size than actual, we cannot have a buffer for the blocks after the
size. After we trncated or extended the file, we are certain that we
don't have a buffer for unknown blocks.

If there's no longer such a buggy kernel, we can rely on lseek() only
when InRecovery. If we had synchronized file size cache we could rely
on the cache even while !InRecovery. (I'm not sure about how vacuum
affects, though.)

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#133Thomas Munro
thomas.munro@gmail.com
In reply to: Kyotaro Horiguchi (#132)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Oct 22, 2020 at 7:33 PM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Thu, 22 Oct 2020 14:16:37 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

smgrtruncate and msgrextend modifies that cache from their parameter,
not from lseek(). At the very first the value in the cache comes from
lseek() but if nothing other than postgres have changed the file size,
I believe we can rely on the cache even with such a buggy kernels even
if still exists.

Mmm. Not exact. The requirement here is that we must be certain that
the we don't have a buffuer for blocks after the file size known to
the process. While recoverying, If the first lseek() returned smaller
size than actual, we cannot have a buffer for the blocks after the
size. After we trncated or extended the file, we are certain that we
don't have a buffer for unknown blocks.

Thanks, I understand now. Something feels fragile about it, perhaps
because it's not really acting as a "cache" anymore despite its name,
but I see the logic now. It becomes the authoritative source of
information, even if the kernel decides to make our file smaller
asynchronously.

If there's no longer such a buggy kernel, we can rely on lseek() only
when InRecovery. If we had synchronized file size cache we could rely
on the cache even while !InRecovery. (I'm not sure about how vacuum
affects, though.)

Perhaps the buggy kernel of 2006 is actually Linux working as designed
according to its philosophy on ejecting dirty buffers on writeback
failure (and apparently adjusting the size at the same time). At
least in 2020 it'll tell us about the problem that caused that when we
next perform an operation that reads the error counter, but in the
case of a relation we're dropping -- the use case in this thread --
that won't happen! (I mean, something else will probably tell you
your system is toast pretty soon, but this particular condition may be
undetected).

I think a synchronised file size cache wouldn't be enough to use this
trick outside the recovery process, because the initial value would
come from a call to lseek(), but unlike recovery, that wouldn't happen
*before* we start putting pages in the buffer pool. Also, if we one
day have a size-limited relcache, even recovery could get into
trouble, if it evicts the RelationData that holds the authoritative
nblocks value.

#134Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Thomas Munro (#130)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 22 Oct 2020 18:54:43 +1300, Thomas Munro <thomas.munro@gmail.com> wrote in

On Thu, Oct 22, 2020 at 5:52 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Per the referenced bug-reporting thread, it was ReiserFS and/or NFS on
SLES 9.3; so, dubious storage choices on an ancient-even-then Linux
kernel.

Ohhhh. I can reproduce that on a modern Linux box by forcing
writeback to a full NFS filesystem[1], approximately as the kernel
does asynchronously when it feels like it, causing the size reported
by SEEK_END to go down.

<test code>

$ cc magic_shrinking_file.c
$ ./a.out
lseek(..., SEEK_END) = 9670656
write(...) = 8192
lseek(..., SEEK_END) = 9678848
fsync(...) = -1
lseek(..., SEEK_END) = 9670656

Interesting..

I think the takeaway point is not so much that that particular bug
might recur as that storage infrastructure does sometimes have bugs.
If you're wanting to introduce new assumptions about what the filesystem
will do, it's prudent to think about how badly will we break if the
assumptions fail.

Yeah. My point was just that the caching trick doesn't seem to
improve matters on this particular front, it can just cache a bogus
value.

[1] /messages/by-id/CAEepm=1FGo=ACPKRmAxvb53mBwyVC=TDwTE0DMzkWjdbAYw7sw@mail.gmail.com

As I wrote in another branch of this thread, the requirement here is
making sure that we don't have a buffer for blocks after the file size
known to the process. Even if the cache gets a bogus value at the
first load, it's still true that we don't have a buffers for blocks
after that size. There's no problem as far as DropRelFileNodeBuffers
doesn't get a smaller value from smgrnblocks than the size the server
thinks.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#135tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Thomas Munro (#133)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Thomas Munro <thomas.munro@gmail.com>

On Thu, Oct 22, 2020 at 7:33 PM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Mmm. Not exact. The requirement here is that we must be certain that
the we don't have a buffuer for blocks after the file size known to
the process. While recoverying, If the first lseek() returned smaller
size than actual, we cannot have a buffer for the blocks after the
size. After we trncated or extended the file, we are certain that we
don't have a buffer for unknown blocks.

Thanks, I understand now. Something feels fragile about it, perhaps
because it's not really acting as a "cache" anymore despite its name,
but I see the logic now. It becomes the authoritative source of
information, even if the kernel decides to make our file smaller
asynchronously.

Thank you Horiguchi-san, you are a savior! I was worried like the end of the world has come.

I think a synchronised file size cache wouldn't be enough to use this
trick outside the recovery process, because the initial value would
come from a call to lseek(), but unlike recovery, that wouldn't happen
*before* we start putting pages in the buffer pool. Also, if we one
day have a size-limited relcache, even recovery could get into
trouble, if it evicts the RelationData that holds the authoritative
nblocks value.

That's too bad, because we hoped we may be able to various operations during normal operation (TRUNCATE, DROP TABLE/INDEX, DROP DATABASE, etc.) An honest man can't believe the system call, that's a hell.

I'm probably being silly, but can't we avoid the problem by using fstat() instead of lseek(SEEK_END)? Would they return the same value from the i-node?

Or, can't we just try to do BufTableLookup() one block after what smgrnblocks() returns?

Regards
Takayuki Tsunakawa

#136Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#135)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 22 Oct 2020 07:31:55 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Thomas Munro <thomas.munro@gmail.com>

On Thu, Oct 22, 2020 at 7:33 PM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Mmm. Not exact. The requirement here is that we must be certain that
the we don't have a buffuer for blocks after the file size known to
the process. While recoverying, If the first lseek() returned smaller
size than actual, we cannot have a buffer for the blocks after the
size. After we trncated or extended the file, we are certain that we
don't have a buffer for unknown blocks.

Thanks, I understand now. Something feels fragile about it, perhaps
because it's not really acting as a "cache" anymore despite its name,
but I see the logic now. It becomes the authoritative source of
information, even if the kernel decides to make our file smaller
asynchronously.

Thank you Horiguchi-san, you are a savior! I was worried like the end of the world has come.

I think a synchronised file size cache wouldn't be enough to use this
trick outside the recovery process, because the initial value would
come from a call to lseek(), but unlike recovery, that wouldn't happen
*before* we start putting pages in the buffer pool. Also, if we one
day have a size-limited relcache, even recovery could get into
trouble, if it evicts the RelationData that holds the authoritative
nblocks value.

That's too bad, because we hoped we may be able to various operations during normal operation (TRUNCATE, DROP TABLE/INDEX, DROP DATABASE, etc.) An honest man can't believe the system call, that's a hell.

I'm probably being silly, but can't we avoid the problem by using fstat() instead of lseek(SEEK_END)? Would they return the same value from the i-node?

Or, can't we just try to do BufTableLookup() one block after what smgrnblocks() returns?

Lossy smgrrelcache or relacache is not a hard obstacle. As the same
with the case of !accurate, we just give up the optimized dropping if
the relcache doesn't give the authoritative size.

By the way, heap scan finds the size of target relation using
smgrnblocks(). I'm not sure why we don't miss recently-extended pages
on a heap-scan? It seems to be possible that concurrent checkpoint
fsyncs relation files inbetween the extension and scanning and the
scanning gets smaller size than it really is.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#137Thomas Munro
thomas.munro@gmail.com
In reply to: Kyotaro Horiguchi (#136)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Oct 22, 2020 at 9:50 PM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

By the way, heap scan finds the size of target relation using
smgrnblocks(). I'm not sure why we don't miss recently-extended pages
on a heap-scan? It seems to be possible that concurrent checkpoint
fsyncs relation files inbetween the extension and scanning and the
scanning gets smaller size than it really is.

Yeah. That's a narrow window: fsync() returns an error after the file
shrinks and we immediately panic. A version with a wider window: the
kernel tries to write in the background, gets an I/O error, shrinks
the file, but we don't know this and we continue running until the
next checkpoint calls fsync(), sees the error and panics. Seq scans
between those two events fail to see recently committed data at the
end of the table.

#138Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#136)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Oct 22, 2020 at 2:20 PM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Thu, 22 Oct 2020 07:31:55 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Thomas Munro <thomas.munro@gmail.com>

On Thu, Oct 22, 2020 at 7:33 PM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Mmm. Not exact. The requirement here is that we must be certain that
the we don't have a buffuer for blocks after the file size known to
the process. While recoverying, If the first lseek() returned smaller
size than actual, we cannot have a buffer for the blocks after the
size. After we trncated or extended the file, we are certain that we
don't have a buffer for unknown blocks.

Thanks, I understand now. Something feels fragile about it, perhaps
because it's not really acting as a "cache" anymore despite its name,
but I see the logic now. It becomes the authoritative source of
information, even if the kernel decides to make our file smaller
asynchronously.

I understand your hesitation but I guess if we can't rely on this
cache in recovery then probably we have a problem without this patch
itself because the current relation extension (in ReadBuffer_common)
relies on the smgrnblocks. So, if the cache lies with us it will
overwrite some existing block.

Thank you Horiguchi-san, you are a savior! I was worried like the end of the world has come.

I think a synchronised file size cache wouldn't be enough to use this
trick outside the recovery process, because the initial value would
come from a call to lseek(), but unlike recovery, that wouldn't happen
*before* we start putting pages in the buffer pool.

This is true because the other sessions might have pulled the page to
buffer pool but I think if we have invalidations for
extension/truncation of a relation then probably before relying on
this value we can process the invalidations to update this cache
value.

Also, if we one
day have a size-limited relcache, even recovery could get into
trouble, if it evicts the RelationData that holds the authoritative
nblocks value.

That's too bad, because we hoped we may be able to various operations during normal operation (TRUNCATE, DROP TABLE/INDEX, DROP DATABASE, etc.) An honest man can't believe the system call, that's a hell.

I'm probably being silly, but can't we avoid the problem by using fstat() instead of lseek(SEEK_END)? Would they return the same value from the i-node?

Or, can't we just try to do BufTableLookup() one block after what smgrnblocks() returns?

Lossy smgrrelcache or relacache is not a hard obstacle. As the same
with the case of !accurate, we just give up the optimized dropping if
the relcache doesn't give the authoritative size.

I think detecting lossy cache is the key thing, probably it might not
be as straight forward as it is in recovery path.

By the way, heap scan finds the size of target relation using
smgrnblocks(). I'm not sure why we don't miss recently-extended pages
on a heap-scan? It seems to be possible that concurrent checkpoint
fsyncs relation files inbetween the extension and scanning and the
scanning gets smaller size than it really is.

Yeah, I think that would be a problem but not as serious as in the
case we are trying to deal here.

--
With Regards,
Amit Kapila.

#139Thomas Munro
thomas.munro@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#135)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Oct 22, 2020 at 8:32 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

I'm probably being silly, but can't we avoid the problem by using fstat() instead of lseek(SEEK_END)? Would they return the same value from the i-node?

Amazingly, st_size can disagree with SEEK_END when using the Linux NFS
client, but its behaviour is worse. Here's a sequence from a Linux
NFS client talking to a Linux NFS server with no free space. This
time, I also replaced the fsync() with sleep(60), just to make it
clear that SEEK_END offset can move at any time due to asynchronous
activity in kernel threads:

lseek(..., SEEK_END) = 9670656
fstat(...) = 0, st_size = 9670656

write(...) = 8192
lseek(..., SEEK_END) = 9678848
fstat(...) = 0, st_size = 9670656 (*1)

sleep(...) = 0

lseek(..., SEEK_END) = 9670656 (*2)
fstat(...) = 0, st_size = 9670656

fsync(...) = -1
lseek(..., SEEK_END) = 9670656
fstat(...) = 0, st_size = 9670656
fsync(...) = 0

However, I'm not entirely sure which phenomena visible here to blame
on which subsystems, and therefore which things to expect on local
filesystems, or on other operating systems. I can say that with a
FreeBSD NFS client and the same Linux NFS server, I don't see
phenomenon *1 (unsurprising) but I do see phenomenon *2 (surprising to
me).

Or, can't we just try to do BufTableLookup() one block after what smgrnblocks() returns?

Unfortunately the problem isn't limited to one block.

#140tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Thomas Munro (#139)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Thomas Munro <thomas.munro@gmail.com>

I'm probably being silly, but can't we avoid the problem by using fstat()

instead of lseek(SEEK_END)? Would they return the same value from the
i-node?

Amazingly, st_size can disagree with SEEK_END when using the Linux NFS
client, but its behaviour is worse. Here's a sequence from a Linux
NFS client talking to a Linux NFS server with no free space. This
time, I also replaced the fsync() with sleep(60), just to make it
clear that SEEK_END offset can move at any time due to asynchronous
activity in kernel threads:

Thank you for experimenting. That's surely amazing. So, it makes sense for commercial DBMSs and MySQL to preallocate data files... (But IIRC, MySQL has provided an option to allocate a file per table like Postgres relatively recently.)

FWIW, it seems safe to use the nodelalloc mount option with ext4 to disable delayed allocation, while xfs doesn't have such an option.

Or, can't we just try to do BufTableLookup() one block after what

smgrnblocks() returns?

Unfortunately the problem isn't limited to one block.

You're right. The data file can be extended by multiple blocks between disk writes.

Regards
Takayuki Tsunakawa

#141k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#140)
4 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi everyone,

Attached are the updated set of patches (V28).
0004 - Truncate optimization is a new patch, while the rest are similar to V27.
This passes the build, regression and TAP tests.

Apologies for the delay.
I'll post the benchmark test results on SSD soon, considering the suggested benchmark of Horiguchi-san:

Currently BUF_DROP_FULL_SCAN_THRESHOLD is set to Nbuffers / 512,
which is quite arbitrary that comes from a wild guess.

Perhaps we need to run benchmarks that drops one relation with several
different ratios between the number of buffers to-be-dropped and Nbuffers,
and preferably both on spinning rust and SSD.

Regards,
Kirk Jamison

Attachments:

v28-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v28-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From 05c70d92a3fba50f4e04546811fe491e3ceb7dc0 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v28 1/4] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v28-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v28-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From 771dc4d608c49316397f8b73d7da677a7675c1e9 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v28 2/4] Add bool param in smgrnblocks() for cached blocks.

---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 15 +++++++++++++--
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 29 insertions(+), 17 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 7e915bc..46fd97d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index d538f257..3874ff3 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -721,7 +721,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 3eee86a..9ab5e74 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -740,7 +740,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2857,7 +2857,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..aad6e5d 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -553,16 +553,27 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
  *					 supplied relation.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  The cached values
+	 * could be smaller than the actual number of existing buffers of the file.
+	 * This is caused by lseek of buggy Linux kernels that might not have
+	 * accounted for the recent write.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (accurate != NULL)
+			*accurate = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
+
+	if (accurate != NULL)
+		*accurate = false;
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..af11b53 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *accurate);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v28-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v28-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 7609877ed5e56a13f847103dd787c04ed7481ce1 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 16 Oct 2020 02:23:05 +0000
Subject: [PATCH v28 3/4] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning. This improves the
performance especially when VACUUM or autovacuum truncated off any
of the empty pages at the end of relation.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 95 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     |  2 +-
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 88 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 9ab5e74..18d8512 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2967,18 +2969,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are known to be accurate. See smgrnblocks() for
+ *		more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		accurate;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2992,10 +3004,76 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of to-be-invalidated blocks of a relation as well
+	 * as the total blocks for a given fork.  The cached value returned by
+	 * smgrnblocks could be smaller than the actual number of existing buffers
+	 * of the file.  This is caused by buggy Linux kernels that might not have
+	 * accounted for the recent write.  Give up the optimization if the block
+	 * count of any fork cannot be trusted.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &accurate);
+
+		if (!accurate)
+			break;
+
+		/* Get the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffer in the hashtable if the block size is known to
+	 * be accurate and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (accurate && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		curBlock;
+
+			for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j]; curBlock++)
+			{
+				uint32		bufHash;		/* hash value for tag */
+				BufferTag	bufTag;			/* identity of requested block */
+				LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				bufHash = BufTableHashCode(&bufTag);
+				bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing. */
+				LWLockAcquire(bufPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&bufTag, bufHash);
+				LWLockRelease(bufPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= firstDelBlock[j])
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3246,8 +3324,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index aad6e5d..6b8528e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -601,7 +601,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v28-0004-TRUNCATE-optimization.patchapplication/octet-stream; name=v28-0004-TRUNCATE-optimization.patchDownload
From 75a220fc3a4919e4e6fa188515f80082b9dacd09 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 28 Oct 2020 06:32:01 +0000
Subject: [PATCH v28 4/4] TRUNCATE optimization.

---
 src/backend/storage/buffer/bufmgr.c | 74 +++++++++++++++++++++++++++++++++++--
 src/backend/storage/smgr/smgr.c     | 14 +++----
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 79 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 18d8512..e1c1190 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3121,17 +3121,37 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
+DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 {
 	int			i,
+				j,
+				nforks,
 				n = 0;
+	SMgrRelation	*rels;
+	RelFileNodeBackend *rnodes;
 	RelFileNode *nodes;
+	BlockNumber	firstDelBlocks[MAX_FORKNUM + 1];
+	BlockNumber	nblocks;
+	BlockNumber	nBlocksToInvalidate;
+	ForkNumber	forks[MAX_FORKNUM + 1];
+	bool		accurate;
 	bool		use_bsearch;
 
 	if (nnodes == 0)
 		return;
 
-	nodes = palloc(sizeof(RelFileNode) * nnodes);	/* non-local relations */
+	/* Create an array which contains all relations to be dropped */
+	rnodes = palloc(sizeof(RelFileNodeBackend) * nnodes);
+	for (i = 0; i < nnodes; i++)
+	{
+		RelFileNodeBackend rnode = smgr_reln[i]->smgr_rnode;
+
+		rnodes[i] = rnode;
+	}
+
+	/* non-local relations */
+	rels = (SMgrRelation *)palloc(nnodes * sizeof(SMgrRelation));
+	nodes = palloc(sizeof(RelFileNode) * nnodes);
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	for (i = 0; i < nnodes; i++)
@@ -3142,7 +3162,10 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 				DropRelFileNodeAllLocalBuffers(rnodes[i].node);
 		}
 		else
+		{
+			rels[n] = smgr_reln[i];
 			nodes[n++] = rnodes[i].node;
+		}
 	}
 
 	/*
@@ -3152,10 +3175,54 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	if (n == 0)
 	{
 		pfree(nodes);
+		pfree(rels);
+		pfree(rnodes);
 		return;
 	}
 
 	/*
+	 * Zero the array of blocks because these will all be dropped anyway.
+	 * Get the total number of blocks for a given fork per relation.
+	 * Give up the optimization if the block count of any fork of relation
+	 * cannot be trusted.
+	 */
+	memset(firstDelBlocks, 0, sizeof(firstDelBlocks));
+	for (i = 0; i < n; i++)
+	{
+		nforks = 0;
+		nBlocksToInvalidate = 0;
+
+		for (j = 0; j <= MAX_FORKNUM; j++)
+		{
+			if (!smgrexists(rels[i], j))
+				continue;
+
+			/* Get the number of blocks for a relation's fork */
+			nblocks = smgrnblocks(rels[i], j, &accurate);
+
+			if (!accurate)
+				break;
+
+			nBlocksToInvalidate += nblocks;
+
+			forks[nforks++] = j;
+		}
+		if (!accurate || j >= MAX_FORKNUM ||
+			nBlocksToInvalidate >= BUF_DROP_FULL_SCAN_THRESHOLD)
+		{
+			goto buffer_full_scan;
+			return;
+		}
+
+		DropRelFileNodeBuffers(rels[i], forks, nforks, firstDelBlocks);
+	}
+	pfree(nodes);
+	pfree(rels);
+	pfree(rnodes);
+	return;
+
+buffer_full_scan:
+	/*
 	 * For low number of relations to drop just use a simple walk through, to
 	 * save the bsearch overhead. The threshold to use is rather a guess than
 	 * an exactly determined value, as it depends on many factors (CPU and RAM
@@ -3208,8 +3275,9 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 		else
 			UnlockBufHdr(bufHdr, buf_state);
 	}
-
 	pfree(nodes);
+	pfree(rels);
+	pfree(rnodes);
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 6b8528e..fd2cf84 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -392,7 +392,13 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 		return;
 
 	/*
-	 * create an array which contains all relations to be dropped, and close
+	 * Get rid of any remaining buffers for the relations.  bufmgr will just
+	 * drop them without bothering to write the contents.
+	 */
+	DropRelFileNodesAllBuffers(rels, nrels);
+
+	/*
+	 * Create an array which contains all relations to be dropped, and close
 	 * each relation's forks at the smgr level while at it
 	 */
 	rnodes = palloc(sizeof(RelFileNodeBackend) * nrels);
@@ -409,12 +415,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	}
 
 	/*
-	 * Get rid of any remaining buffers for the relations.  bufmgr will just
-	 * drop them without bothering to write the contents.
-	 */
-	DropRelFileNodesAllBuffers(rnodes, nrels);
-
-	/*
 	 * It'd be nice to tell the stats collector to forget them immediately,
 	 * too. But we can't because we don't know the OIDs.
 	 */
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 056f65e..2e5189b 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -205,7 +205,7 @@ extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels)
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
-extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
+extern void DropRelFileNodesAllBuffers(struct SMgrRelationData **smgr_reln, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
 #define RelationGetNumberOfBlocks(reln) \
-- 
1.8.3.1

#142tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#141)
RE: [Patch] Optimize dropping of relation buffers using dlist

The patch looks almost good except for the minor ones:

(1)
+	for (i = 0; i < nnodes; i++)
+	{
+		RelFileNodeBackend rnode = smgr_reln[i]->smgr_rnode;
+
+		rnodes[i] = rnode;
+	}

You can write:

+ for (i = 0; i < nnodes; i++)
+ rnodes[i] = smgr_reln[i]->smgr_rnode;

(2)
+ if (!accurate || j >= MAX_FORKNUM ||

The correct condition would be:

+ if (j <= MAX_FORKNUM ||

because j becomes MAX_FORKNUM + 1 if accurate sizes for all forks could be obtained. If any fork's size is inaccurate, j is <= MAX_FORKNUM when exiting the loop, so you don't need to test for accurate flag.

(3)
+		{
+			goto buffer_full_scan;
+			return;
+		}

return after goto cannot be reached, so this should just be:

+ goto buffer_full_scan;

Regards
Takayuki Tsunakawa

#143k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#142)
4 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi,

I've updated the patch 0004 (Truncate optimization) with the previous comments of
Tsunakawa-san already addressed in the patch. (Thank you very much for the review.)
The change here compared to the previous version is that in DropRelFileNodesAllBuffers()
we don't check for the accurate flag anymore when deciding whether to optimize or not.
For relations with blocks that do not exceed the threshold for full scan, we call
DropRelFileNodeBuffers where the flag will be checked anyway. Otherwise, we proceed
to the traditional buffer scan. Thoughts?

I've done recovery performance for TRUNCATE.
Test case: 1 parent table with 100 child partitions. TRUNCATE each child partition (1 transaction per table).
Currently, it takes a while to recover when we have large shared_buffers setting, but with the patch applied
the recovery is almost constant (0.206 s below).

| s_b | master | patched |
|-------|--------|---------|
| 128MB | 0.105 | 0.105 |
| 1GB | 0.205 | 0.205 |
| 20GB | 2.008 | 0.206 |
| 100GB | 9.315 | 0.206 |

Method of Testing (assuming streaming replication is configured):
1. Create 1 parent table and 100 child partitions
2. Insert data to each table.
3. Pause WAL replay on standby. ( SELECT pg_wal_replay_pause(); )
4. TRUNCATE each child partitions on primary (1 transaction per table). Stop the primary
5. Resume the WAL replay and promote standby. ( SELECT pg_wal_replay_resume(); pg_ctl promote)
I have confirmed that the relations became empty on standby.

Your thoughts, feedback are very much appreciated.

Regards,
Kirk Jamison

Attachments:

v29-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v29-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From c866a4024ab8ae03f31e6c0e4e78c589c2f789d6 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v29 1/4] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v29-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v29-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From b2ad567d000aff5815880194dc8db14545ea5a33 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v29 2/4] Add bool param in smgrnblocks() for cached blocks.

---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 15 +++++++++++++--
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 29 insertions(+), 17 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 7e915bc..46fd97d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index d538f257..3874ff3 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -721,7 +721,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 0adf048..ef5d223 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -740,7 +740,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2857,7 +2857,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..aad6e5d 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -553,16 +553,27 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
  *					 supplied relation.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  The cached values
+	 * could be smaller than the actual number of existing buffers of the file.
+	 * This is caused by lseek of buggy Linux kernels that might not have
+	 * accounted for the recent write.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (accurate != NULL)
+			*accurate = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
+
+	if (accurate != NULL)
+		*accurate = false;
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..af11b53 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *accurate);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v29-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v29-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 35bd2c3b4642ac3af44dfa162c404d8752684675 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 16 Oct 2020 02:23:05 +0000
Subject: [PATCH v29 3/4] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning. This improves the
performance especially when VACUUM or autovacuum truncated off any
of the empty pages at the end of relation.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 95 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     |  2 +-
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 88 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ef5d223..623fe6e 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 512)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -2967,18 +2969,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are known to be accurate. See smgrnblocks() for
+ *		more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		accurate;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+	BufferDesc	*bufHdr;
+	uint32		buf_state;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2992,10 +3004,76 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of to-be-invalidated blocks of a relation as well
+	 * as the total blocks for a given fork.  The cached value returned by
+	 * smgrnblocks could be smaller than the actual number of existing buffers
+	 * of the file.  This is caused by buggy Linux kernels that might not have
+	 * accounted for the recent write.  Give up the optimization if the block
+	 * count of any fork cannot be trusted.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &accurate);
+
+		if (!accurate)
+			break;
+
+		/* Get the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffer in the hashtable if the block size is known to
+	 * be accurate and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (accurate && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			BlockNumber		curBlock;
+
+			for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j]; curBlock++)
+			{
+				uint32		bufHash;		/* hash value for tag */
+				BufferTag	bufTag;			/* identity of requested block */
+				LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+				int		buf_id;
+
+				/* create a tag so we can lookup the buffer */
+				INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+				/* determine its hash code and partition lock ID */
+				bufHash = BufTableHashCode(&bufTag);
+				bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+				/* Check that it is in the buffer pool. If not, do nothing. */
+				LWLockAcquire(bufPartitionLock, LW_SHARED);
+				buf_id = BufTableLookup(&bufTag, bufHash);
+				LWLockRelease(bufPartitionLock);
+
+				if (buf_id < 0)
+					continue;
+
+				bufHdr = GetBufferDescriptor(buf_id);
+
+				buf_state = LockBufHdr(bufHdr);
+
+				if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+					bufHdr->tag.forkNum == forkNum[j] &&
+					bufHdr->tag.blockNum >= firstDelBlock[j])
+					InvalidateBuffer(bufHdr);	/* releases spinlock */
+				else
+					UnlockBufHdr(bufHdr, buf_state);
+			}
+		}
+		return;
+	}
 	for (i = 0; i < NBuffers; i++)
 	{
-		BufferDesc *bufHdr = GetBufferDescriptor(i);
-		uint32		buf_state;
+		bufHdr = GetBufferDescriptor(i);
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -3246,8 +3324,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index aad6e5d..6b8528e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -601,7 +601,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index a21cab2..37093fb 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v29-0004-TRUNCATE-optimization.patchapplication/octet-stream; name=v29-0004-TRUNCATE-optimization.patchDownload
From f5ef702ffaeeaa3ca11ecd042abc9b63b5b6db83 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 28 Oct 2020 06:32:01 +0000
Subject: [PATCH v29 4/4] TRUNCATE optimization.

DropRelFileNodesAllBuffers() is optimized to skip the time-consuming
scan of the whole buffer pool when the relation is small enough, or
when the number of blocks to be invalidated is below the full scan
threshold. This improves the performance when the TRUNCATE command
truncated off any of the empty pages at the end of relation.
---
 src/backend/storage/buffer/bufmgr.c | 63 +++++++++++++++++++++++++++++++++++--
 src/backend/storage/smgr/smgr.c     | 14 ++++-----
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 68 insertions(+), 11 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 623fe6e..e3ed565 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3121,17 +3121,32 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
+DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 {
 	int			i,
+				j,
+				nforks,
 				n = 0;
+	SMgrRelation	*rels;
+	RelFileNodeBackend *rnodes;
 	RelFileNode *nodes;
+	BlockNumber	nblocks;
+	BlockNumber	nBlocksToInvalidate;
+	BlockNumber	firstDelBlocks[MAX_FORKNUM + 1];
+	ForkNumber	forks[MAX_FORKNUM + 1];
 	bool		use_bsearch;
 
 	if (nnodes == 0)
 		return;
 
-	nodes = palloc(sizeof(RelFileNode) * nnodes);	/* non-local relations */
+	/* Create an array which contains all relations to be dropped */
+	rnodes = palloc(sizeof(RelFileNodeBackend) * nnodes);
+	for (i = 0; i < nnodes; i++)
+		rnodes[i] = smgr_reln[i]->smgr_rnode;
+
+	/* non-local relations */
+	rels = (SMgrRelation *)palloc(nnodes * sizeof(SMgrRelation));
+	nodes = palloc(sizeof(RelFileNode) * nnodes);
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	for (i = 0; i < nnodes; i++)
@@ -3142,7 +3157,10 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 				DropRelFileNodeAllLocalBuffers(rnodes[i].node);
 		}
 		else
+		{
+			rels[n] = smgr_reln[i];
 			nodes[n++] = rnodes[i].node;
+		}
 	}
 
 	/*
@@ -3152,10 +3170,48 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	if (n == 0)
 	{
 		pfree(nodes);
+		pfree(rels);
+		pfree(rnodes);
 		return;
 	}
 
 	/*
+	 * Zero the array of blocks because these will all be dropped anyway.
+	 * We don't care if the block count of relation fork cannot be trusted,
+	 * as it will be checked in DropRelFileNodeBuffers.  However, we give
+	 * up the optimization if the relation is large enough that the number
+	 * of blocks to be invalidated exceeds the threshold for full scan.
+	 */
+	memset(firstDelBlocks, 0, sizeof(firstDelBlocks));
+	for (i = 0; i < n; i++)
+	{
+		nforks = 0;
+		nBlocksToInvalidate = 0;
+
+		for (j = 0; j <= MAX_FORKNUM; j++)
+		{
+			if (!smgrexists(rels[i], j))
+				continue;
+
+			/* Get the number of blocks for a relation's fork */
+			nblocks = smgrnblocks(rels[i], j, NULL);
+
+			nBlocksToInvalidate += nblocks;
+
+			forks[nforks++] = j;
+		}
+		if (nBlocksToInvalidate >= BUF_DROP_FULL_SCAN_THRESHOLD)
+			goto buffer_full_scan;
+
+		DropRelFileNodeBuffers(rels[i], forks, nforks, firstDelBlocks);
+	}
+	pfree(nodes);
+	pfree(rels);
+	pfree(rnodes);
+	return;
+
+buffer_full_scan:
+	/*
 	 * For low number of relations to drop just use a simple walk through, to
 	 * save the bsearch overhead. The threshold to use is rather a guess than
 	 * an exactly determined value, as it depends on many factors (CPU and RAM
@@ -3208,8 +3264,9 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 		else
 			UnlockBufHdr(bufHdr, buf_state);
 	}
-
 	pfree(nodes);
+	pfree(rels);
+	pfree(rnodes);
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 6b8528e..fd2cf84 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -392,7 +392,13 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 		return;
 
 	/*
-	 * create an array which contains all relations to be dropped, and close
+	 * Get rid of any remaining buffers for the relations.  bufmgr will just
+	 * drop them without bothering to write the contents.
+	 */
+	DropRelFileNodesAllBuffers(rels, nrels);
+
+	/*
+	 * Create an array which contains all relations to be dropped, and close
 	 * each relation's forks at the smgr level while at it
 	 */
 	rnodes = palloc(sizeof(RelFileNodeBackend) * nrels);
@@ -409,12 +415,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	}
 
 	/*
-	 * Get rid of any remaining buffers for the relations.  bufmgr will just
-	 * drop them without bothering to write the contents.
-	 */
-	DropRelFileNodesAllBuffers(rnodes, nrels);
-
-	/*
 	 * It'd be nice to tell the stats collector to forget them immediately,
 	 * too. But we can't because we don't know the OIDs.
 	 */
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 37093fb..0cc40e4 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -205,7 +205,7 @@ extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels)
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
-extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
+extern void DropRelFileNodesAllBuffers(struct SMgrRelationData **smgr_reln, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
 #define RelationGetNumberOfBlocks(reln) \
-- 
1.8.3.1

#144Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#143)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Nov 4, 2020 at 8:28 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Hi,

I've updated the patch 0004 (Truncate optimization) with the previous comments of
Tsunakawa-san already addressed in the patch. (Thank you very much for the review.)
The change here compared to the previous version is that in DropRelFileNodesAllBuffers()
we don't check for the accurate flag anymore when deciding whether to optimize or not.
For relations with blocks that do not exceed the threshold for full scan, we call
DropRelFileNodeBuffers where the flag will be checked anyway. Otherwise, we proceed
to the traditional buffer scan. Thoughts?

Can we do a Truncate optimization once we decide about your other
patch as I see a few problems with it? If we can get the first patch
(vacuum optimization) committed it might be a bit easier for us to get
the truncate optimization. If possible, let's focus on (auto)vacuum
optimization first.

Few comments on patches:
======================
v29-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks
-----------------------------------------------------------------------------------
1.
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
 {
  BlockNumber result;
  /*
  * For now, we only use cached values in recovery due to lack of a shared
- * invalidation mechanism for changes in file size.
+ * invalidation mechanism for changes in file size.  The cached values
+ * could be smaller than the actual number of existing buffers of the file.
+ * This is caused by lseek of buggy Linux kernels that might not have
+ * accounted for the recent write.
  */
  if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+ {
+ if (accurate != NULL)
+ *accurate = true;
+

I don't understand this comment. Few emails back, I think we have
discussed that cached value can't be less than the number of buffers
during recovery. If that happens to be true then we have some problem.
If you want to explain 'accurate' variable then you can do the same
atop of function. Would it be better to name this variable as
'cached'?

v29-0003-Optimize-DropRelFileNodeBuffers-during-recovery
----------------------------------------------------------------------------------
2.
+ /* Check that it is in the buffer pool. If not, do nothing. */
+ LWLockAcquire(bufPartitionLock, LW_SHARED);
+ buf_id = BufTableLookup(&bufTag, bufHash);
+ LWLockRelease(bufPartitionLock);
+
+ if (buf_id < 0)
+ continue;
+
+ bufHdr = GetBufferDescriptor(buf_id);
+
+ buf_state = LockBufHdr(bufHdr);
+
+ if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&

I think a pre-check for RelFileNode might be better before LockBufHdr
for the reasons mentioned in this function few lines down.

3.
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
     int nforks, BlockNumber *firstDelBlock)
 {
  int i;
  int j;
+ RelFileNodeBackend rnode;
+ bool accurate;

It is better to initialize accurate with false. Again, is it better to
change this variable name as 'cached'.

4.
+ /*
+ * Look up the buffer in the hashtable if the block size is known to
+ * be accurate and the total blocks to be invalidated is below the
+ * full scan threshold.  Otherwise, give up the optimization.
+ */
+ if (accurate && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+ {
+ for (j = 0; j < nforks; j++)
+ {
+ BlockNumber curBlock;
+
+ for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j]; curBlock++)
+ {
+ uint32 bufHash; /* hash value for tag */
+ BufferTag bufTag; /* identity of requested block */
+ LWLock     *bufPartitionLock; /* buffer partition lock for it */
+ int buf_id;
+
+ /* create a tag so we can lookup the buffer */
+ INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+ /* determine its hash code and partition lock ID */
+ bufHash = BufTableHashCode(&bufTag);
+ bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+ /* Check that it is in the buffer pool. If not, do nothing. */
+ LWLockAcquire(bufPartitionLock, LW_SHARED);
+ buf_id = BufTableLookup(&bufTag, bufHash);
+ LWLockRelease(bufPartitionLock);
+
+ if (buf_id < 0)
+ continue;
+
+ bufHdr = GetBufferDescriptor(buf_id);
+
+ buf_state = LockBufHdr(bufHdr);
+
+ if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+ bufHdr->tag.forkNum == forkNum[j] &&
+ bufHdr->tag.blockNum >= firstDelBlock[j])
+ InvalidateBuffer(bufHdr); /* releases spinlock */
+ else
+ UnlockBufHdr(bufHdr, buf_state);
+ }
+ }
+ return;
+ }

Can we move the code under this 'if' condition to a separate function,
say FindAndDropRelFileNodeBuffers or something like that?

v29-0004-TRUNCATE-optimization
------------------------------------------------
5.
+ for (i = 0; i < n; i++)
+ {
+ nforks = 0;
+ nBlocksToInvalidate = 0;
+
+ for (j = 0; j <= MAX_FORKNUM; j++)
+ {
+ if (!smgrexists(rels[i], j))
+ continue;
+
+ /* Get the number of blocks for a relation's fork */
+ nblocks = smgrnblocks(rels[i], j, NULL);
+
+ nBlocksToInvalidate += nblocks;
+
+ forks[nforks++] = j;
+ }
+ if (nBlocksToInvalidate >= BUF_DROP_FULL_SCAN_THRESHOLD)
+ goto buffer_full_scan;
+
+ DropRelFileNodeBuffers(rels[i], forks, nforks, firstDelBlocks);
+ }
+ pfree(nodes);
+ pfree(rels);
+ pfree(rnodes);
+ return;

I think this can be slower than the current Truncate. Say there are
three relations and for one of them the size is greater than
BUF_DROP_FULL_SCAN_THRESHOLD then you would anyway have to scan the
entire shared buffers so the work done in optimized path for other two
relations will add some over head.

Also, as written, I think you need to remove the nodes for which you
have invalidated the buffers via optimized path, no.

--
With Regards,
Amit Kapila.

#145Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#144)
Re: [Patch] Optimize dropping of relation buffers using dlist

Hello.

Many of the quetions are on the code following my past suggestions.

At Wed, 4 Nov 2020 15:59:17 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Wed, Nov 4, 2020 at 8:28 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Hi,

I've updated the patch 0004 (Truncate optimization) with the previous comments of
Tsunakawa-san already addressed in the patch. (Thank you very much for the review.)
The change here compared to the previous version is that in DropRelFileNodesAllBuffers()
we don't check for the accurate flag anymore when deciding whether to optimize or not.
For relations with blocks that do not exceed the threshold for full scan, we call
DropRelFileNodeBuffers where the flag will be checked anyway. Otherwise, we proceed
to the traditional buffer scan. Thoughts?

Can we do a Truncate optimization once we decide about your other
patch as I see a few problems with it? If we can get the first patch
(vacuum optimization) committed it might be a bit easier for us to get
the truncate optimization. If possible, let's focus on (auto)vacuum
optimization first.

Few comments on patches:
======================
v29-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks
-----------------------------------------------------------------------------------
1.
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
{
BlockNumber result;
/*
* For now, we only use cached values in recovery due to lack of a shared
- * invalidation mechanism for changes in file size.
+ * invalidation mechanism for changes in file size.  The cached values
+ * could be smaller than the actual number of existing buffers of the file.
+ * This is caused by lseek of buggy Linux kernels that might not have
+ * accounted for the recent write.
*/
if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+ {
+ if (accurate != NULL)
+ *accurate = true;
+

I don't understand this comment. Few emails back, I think we have
discussed that cached value can't be less than the number of buffers
during recovery. If that happens to be true then we have some problem.
If you want to explain 'accurate' variable then you can do the same
atop of function. Would it be better to name this variable as
'cached'?

(I agree that the comment needs to be fixed.)

FWIW I don't think 'cached' suggests the characteristics of the
returned value on its interface. It was introduced to reduce fseek()
calls, and after that we have found that it can be regarded as the
authoritative source of the file size. The "accurate" means that it
is guaranteed that we don't have a buffer for the file blocks further
than that number. I don't come up with a more proper word than
"accurate" but also I don't think "cached" is proper here.

By the way, if there's a case where we extend a file by more than one
block the cached value becomes invalid. I'm not sure if it actually
happens, but the following sequence may lead to a problem. We need a
protection for that case.

smgrnblocks() : cached n
truncate to n-5 : cached n=5
extend to m + 2 : cached invalid
(fsync failed)
smgrnblocks() : returns and cached n-5

v29-0003-Optimize-DropRelFileNodeBuffers-during-recovery
----------------------------------------------------------------------------------
2.
+ /* Check that it is in the buffer pool. If not, do nothing. */
+ LWLockAcquire(bufPartitionLock, LW_SHARED);
+ buf_id = BufTableLookup(&bufTag, bufHash);
+ LWLockRelease(bufPartitionLock);
+
+ if (buf_id < 0)
+ continue;
+
+ bufHdr = GetBufferDescriptor(buf_id);
+
+ buf_state = LockBufHdr(bufHdr);
+
+ if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&

I think a pre-check for RelFileNode might be better before LockBufHdr
for the reasons mentioned in this function few lines down.

The equivalent check is already done by BufTableLookup(). The last
line in the above is not a precheck but the final check.

3.
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
int nforks, BlockNumber *firstDelBlock)
{
int i;
int j;
+ RelFileNodeBackend rnode;
+ bool accurate;

It is better to initialize accurate with false. Again, is it better to
change this variable name as 'cached'.

*I* agree to initilization.

4.
+ /*
+ * Look up the buffer in the hashtable if the block size is known to
+ * be accurate and the total blocks to be invalidated is below the
+ * full scan threshold.  Otherwise, give up the optimization.
+ */
+ if (accurate && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+ {
+ for (j = 0; j < nforks; j++)
+ {
+ BlockNumber curBlock;
+
+ for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j]; curBlock++)
+ {
+ uint32 bufHash; /* hash value for tag */
+ BufferTag bufTag; /* identity of requested block */
+ LWLock     *bufPartitionLock; /* buffer partition lock for it */
+ int buf_id;
+
+ /* create a tag so we can lookup the buffer */
+ INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+ /* determine its hash code and partition lock ID */
+ bufHash = BufTableHashCode(&bufTag);
+ bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+ /* Check that it is in the buffer pool. If not, do nothing. */
+ LWLockAcquire(bufPartitionLock, LW_SHARED);
+ buf_id = BufTableLookup(&bufTag, bufHash);
+ LWLockRelease(bufPartitionLock);
+
+ if (buf_id < 0)
+ continue;
+
+ bufHdr = GetBufferDescriptor(buf_id);
+
+ buf_state = LockBufHdr(bufHdr);
+
+ if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+ bufHdr->tag.forkNum == forkNum[j] &&
+ bufHdr->tag.blockNum >= firstDelBlock[j])
+ InvalidateBuffer(bufHdr); /* releases spinlock */
+ else
+ UnlockBufHdr(bufHdr, buf_state);
+ }
+ }
+ return;
+ }

Can we move the code under this 'if' condition to a separate function,
say FindAndDropRelFileNodeBuffers or something like that?

Thinking about the TRUNCATE optimization, it sounds reasonable to have
a separate function, which runs the optmized dropping unconditionally.

v29-0004-TRUNCATE-optimization
------------------------------------------------
5.
+ for (i = 0; i < n; i++)
+ {
+ nforks = 0;
+ nBlocksToInvalidate = 0;
+
+ for (j = 0; j <= MAX_FORKNUM; j++)
+ {
+ if (!smgrexists(rels[i], j))
+ continue;
+
+ /* Get the number of blocks for a relation's fork */
+ nblocks = smgrnblocks(rels[i], j, NULL);
+
+ nBlocksToInvalidate += nblocks;
+
+ forks[nforks++] = j;
+ }
+ if (nBlocksToInvalidate >= BUF_DROP_FULL_SCAN_THRESHOLD)
+ goto buffer_full_scan;
+
+ DropRelFileNodeBuffers(rels[i], forks, nforks, firstDelBlocks);
+ }
+ pfree(nodes);
+ pfree(rels);
+ pfree(rnodes);
+ return;

I think this can be slower than the current Truncate. Say there are
BUF_DROP_FULL_SCAN_THRESHOLD then you would anyway have to scan the
entire shared buffers so the work done in optimized path for other two
relations will add some over head.

That's true. The criteria here is the number of blocks of all
relations. And even if all of the relations is smaller than the
threshold, we should go to the full-scan dropping if the total size
exceeds the threshold. So we cannot reuse DropRelFileNodeBuffers() as
is here.

Also, as written, I think you need to remove the nodes for which you
have invalidated the buffers via optimized path, no.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#146k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Kyotaro Horiguchi (#145)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, November 5, 2020 10:22 AM, Horiguchi-san wrote:

Hello.

Many of the quetions are on the code following my past suggestions.

Yeah, I was also about to answer with the feedback you have given.
Thank you for replying and taking a look too.

At Wed, 4 Nov 2020 15:59:17 +0530, Amit Kapila <amit.kapila16@gmail.com>
wrote in

On Wed, Nov 4, 2020 at 8:28 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Hi,

I've updated the patch 0004 (Truncate optimization) with the
previous comments of Tsunakawa-san already addressed in the patch.
(Thank you very much for the review.) The change here compared to
the previous version is that in DropRelFileNodesAllBuffers() we don't

check for the accurate flag anymore when deciding whether to optimize or
not.

For relations with blocks that do not exceed the threshold for full
scan, we call DropRelFileNodeBuffers where the flag will be checked
anyway. Otherwise, we proceed to the traditional buffer scan. Thoughts?

Can we do a Truncate optimization once we decide about your other
patch as I see a few problems with it? If we can get the first patch
(vacuum optimization) committed it might be a bit easier for us to get
the truncate optimization. If possible, let's focus on (auto)vacuum
optimization first.

Sure. That'd be better.

Few comments on patches:
======================
v29-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks
----------------------------------------------------------------------
-------------
1.
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
{
BlockNumber result;
/*
* For now, we only use cached values in recovery due to lack of a
shared
- * invalidation mechanism for changes in file size.
+ * invalidation mechanism for changes in file size.  The cached
+ values
+ * could be smaller than the actual number of existing buffers of the file.
+ * This is caused by lseek of buggy Linux kernels that might not have
+ * accounted for the recent write.
*/
if (InRecovery && reln->smgr_cached_nblocks[forknum] !=
InvalidBlockNumber)
+ {
+ if (accurate != NULL)
+ *accurate = true;
+

I don't understand this comment. Few emails back, I think we have
discussed that cached value can't be less than the number of buffers
during recovery. If that happens to be true then we have some problem.
If you want to explain 'accurate' variable then you can do the same
atop of function. Would it be better to name this variable as
'cached'?

(I agree that the comment needs to be fixed.)

FWIW I don't think 'cached' suggests the characteristics of the returned value
on its interface. It was introduced to reduce fseek() calls, and after that we
have found that it can be regarded as the authoritative source of the file size.
The "accurate" means that it is guaranteed that we don't have a buffer for the
file blocks further than that number. I don't come up with a more proper
word than "accurate" but also I don't think "cached" is proper here.

I also couldn't think of a better parameter name. Accurate seems to be better fit
as it describes a measurement close to an accepted value.
How about fixing the comment like below, would this suffice?

/*
* smgrnblocks() -- Calculate the number of blocks in the
* supplied relation.
*
* accurate flag acts as an authoritative source of the file size and
* ensures that no buffers exist for blocks after the file size is known
* to the process.
*/
BlockNumber
smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
{
BlockNumber result;

/*
* For now, we only use cached values in recovery due to lack of a shared
* invalidation mechanism for changes in file size. In recovery, the cached
* value returned by the first lseek could be smaller than the actual number
* of existing buffers of the file, which is caused by buggy Linux kernels
* that might not have accounted for the recent write. However, we can
* still rely on the cached value even if we get a bogus value from first
* lseek since it is impossible to have buffer for blocks after the file size.
*/

By the way, if there's a case where we extend a file by more than one block the
cached value becomes invalid. I'm not sure if it actually happens, but the
following sequence may lead to a problem. We need a protection for that
case.

smgrnblocks() : cached n
truncate to n-5 : cached n=5
extend to m + 2 : cached invalid
(fsync failed)
smgrnblocks() : returns and cached n-5

I am not sure if the patch should cover this or should be a separate thread altogether since
a number of functions also rely on the smgrnblocks(). But I'll take it into consideration.

v29-0003-Optimize-DropRelFileNodeBuffers-during-recovery
----------------------------------------------------------------------
------------
2.
+ /* Check that it is in the buffer pool. If not, do nothing. */
+ LWLockAcquire(bufPartitionLock, LW_SHARED); buf_id =
+ BufTableLookup(&bufTag, bufHash); LWLockRelease(bufPartitionLock);
+
+ if (buf_id < 0)
+ continue;
+
+ bufHdr = GetBufferDescriptor(buf_id);
+
+ buf_state = LockBufHdr(bufHdr);
+
+ if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&

I think a pre-check for RelFileNode might be better before LockBufHdr
for the reasons mentioned in this function few lines down.

The equivalent check is already done by BufTableLookup(). The last line in
the above is not a precheck but the final check.

Right. So I'll retain that current code.

3.
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber

*forkNum,

+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber

*forkNum,

int nforks, BlockNumber *firstDelBlock) {
int i;
int j;
+ RelFileNodeBackend rnode;
+ bool accurate;

It is better to initialize accurate with false. Again, is it better to
change this variable name as 'cached'.

*I* agree to initilization.

Understood. I'll include only the initialization in the next updated patch.

4.
+ /*
+ * Look up the buffer in the hashtable if the block size is known to
+ * be accurate and the total blocks to be invalidated is below the
+ * full scan threshold.  Otherwise, give up the optimization.
+ */
+ if (accurate && nBlocksToInvalidate <

BUF_DROP_FULL_SCAN_THRESHOLD)

+ { for (j = 0; j < nforks; j++) { BlockNumber curBlock;
+
+ for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j];
+ curBlock++) {
+ uint32 bufHash; /* hash value for tag */ BufferTag bufTag; /*
+ identity of requested block */
+ LWLock     *bufPartitionLock; /* buffer partition lock for it */
+ int buf_id;
+
+ /* create a tag so we can lookup the buffer */
+ INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+ /* determine its hash code and partition lock ID */ bufHash =
+ BufTableHashCode(&bufTag); bufPartitionLock =
+ BufMappingPartitionLock(bufHash);
+
+ /* Check that it is in the buffer pool. If not, do nothing. */
+ LWLockAcquire(bufPartitionLock, LW_SHARED); buf_id =
+ BufTableLookup(&bufTag, bufHash); LWLockRelease(bufPartitionLock);
+
+ if (buf_id < 0)
+ continue;
+
+ bufHdr = GetBufferDescriptor(buf_id);
+
+ buf_state = LockBufHdr(bufHdr);
+
+ if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+ bufHdr->tag.forkNum == forkNum[j] && tag.blockNum >=
+ bufHdr->firstDelBlock[j])
+ InvalidateBuffer(bufHdr); /* releases spinlock */ else
+ UnlockBufHdr(bufHdr, buf_state); } } return; }

Can we move the code under this 'if' condition to a separate function,
say FindAndDropRelFileNodeBuffers or something like that?

Thinking about the TRUNCATE optimization, it sounds reasonable to have a
separate function, which runs the optmized dropping unconditionally.

Hmm, sure., although only DropRelFileNodeBuffers() would call the new function.
I guess it won't be a problem.

v29-0004-TRUNCATE-optimization
------------------------------------------------
5.
+ for (i = 0; i < n; i++)
+ {
+ nforks = 0;
+ nBlocksToInvalidate = 0;
+
+ for (j = 0; j <= MAX_FORKNUM; j++)
+ {
+ if (!smgrexists(rels[i], j))
+ continue;
+
+ /* Get the number of blocks for a relation's fork */ nblocks =
+ smgrnblocks(rels[i], j, NULL);
+
+ nBlocksToInvalidate += nblocks;
+
+ forks[nforks++] = j;
+ }
+ if (nBlocksToInvalidate >= BUF_DROP_FULL_SCAN_THRESHOLD)

goto

+ buffer_full_scan;
+
+ DropRelFileNodeBuffers(rels[i], forks, nforks, firstDelBlocks); }
+ pfree(nodes); pfree(rels); pfree(rnodes); return;

I think this can be slower than the current Truncate. Say there are
BUF_DROP_FULL_SCAN_THRESHOLD then you would anyway have to

scan the

entire shared buffers so the work done in optimized path for other two
relations will add some over head.

That's true. The criteria here is the number of blocks of all relations. And
even if all of the relations is smaller than the threshold, we should go to the
full-scan dropping if the total size exceeds the threshold. So we cannot
reuse DropRelFileNodeBuffers() as is here.

Also, as written, I think you need to remove the nodes for which you
have invalidated the buffers via optimized path, no.

Right, in the current patch it is indeed slower.
But the decision criteria whether to optimize or not is decided per relation,
not for all relations. So there is a possibility that we have already invalidated
buffers of the first relation, but the next relation buffers exceed the threshold that we
need to do the full scan. So yes that should be fixed. Remove the nodes that we
have already invalidated so that we don't scan them anymore when scanning NBuffers.
I will fix in the next version.

Thank you for the helpful feedback. I'll upload the updated set of patches soon
also when we reach a consensus on the boolean parameter name too.

Regards,
Kirk Jamison

#147Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#146)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Nov 5, 2020 at 8:26 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Thursday, November 5, 2020 10:22 AM, Horiguchi-san wrote:

Can we do a Truncate optimization once we decide about your other
patch as I see a few problems with it? If we can get the first patch
(vacuum optimization) committed it might be a bit easier for us to get
the truncate optimization. If possible, let's focus on (auto)vacuum
optimization first.

Sure. That'd be better.

Okay, thanks.

Few comments on patches:
======================
v29-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks
----------------------------------------------------------------------
-------------
1.
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
{
BlockNumber result;
/*
* For now, we only use cached values in recovery due to lack of a
shared
- * invalidation mechanism for changes in file size.
+ * invalidation mechanism for changes in file size.  The cached
+ values
+ * could be smaller than the actual number of existing buffers of the file.
+ * This is caused by lseek of buggy Linux kernels that might not have
+ * accounted for the recent write.
*/
if (InRecovery && reln->smgr_cached_nblocks[forknum] !=
InvalidBlockNumber)
+ {
+ if (accurate != NULL)
+ *accurate = true;
+

I don't understand this comment. Few emails back, I think we have
discussed that cached value can't be less than the number of buffers
during recovery. If that happens to be true then we have some problem.
If you want to explain 'accurate' variable then you can do the same
atop of function. Would it be better to name this variable as
'cached'?

(I agree that the comment needs to be fixed.)

FWIW I don't think 'cached' suggests the characteristics of the returned value
on its interface. It was introduced to reduce fseek() calls, and after that we
have found that it can be regarded as the authoritative source of the file size.
The "accurate" means that it is guaranteed that we don't have a buffer for the
file blocks further than that number. I don't come up with a more proper
word than "accurate" but also I don't think "cached" is proper here.

Sure but that is not the guarantee this API gives. It has to be
guaranteed by the logic else-where, so not sure if it is a good idea
to try to reflect the same here. The comments in the caller where we
use this should explain why it is safe to use this value.

I also couldn't think of a better parameter name. Accurate seems to be better fit
as it describes a measurement close to an accepted value.
How about fixing the comment like below, would this suffice?

/*
* smgrnblocks() -- Calculate the number of blocks in the
* supplied relation.
*
* accurate flag acts as an authoritative source of the file size and
* ensures that no buffers exist for blocks after the file size is known
* to the process.
*/
BlockNumber
smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
{
BlockNumber result;

/*
* For now, we only use cached values in recovery due to lack of a shared
* invalidation mechanism for changes in file size. In recovery, the cached
* value returned by the first lseek could be smaller than the actual number
* of existing buffers of the file, which is caused by buggy Linux kernels
* that might not have accounted for the recent write. However, we can
* still rely on the cached value even if we get a bogus value from first
* lseek since it is impossible to have buffer for blocks after the file size.
*/

By the way, if there's a case where we extend a file by more than one block the
cached value becomes invalid. I'm not sure if it actually happens, but the
following sequence may lead to a problem. We need a protection for that
case.

smgrnblocks() : cached n
truncate to n-5 : cached n=5
extend to m + 2 : cached invalid
(fsync failed)
smgrnblocks() : returns and cached n-5

I think one possible idea is to actually commit the Assert patch
(v29-0001-Prevent-invalidating-blocks-in-smgrextend-during) to ensure
that it can't happen during recovery. And even if it happens why would
there be any buffer with the block in it left when the fsync failed?
And if there is no buffer with a block which doesn't account due to
lseek lies then there shouldn't be any problem. Do you have any other
ideas on what better can be done here?

I am not sure if the patch should cover this or should be a separate thread altogether since
a number of functions also rely on the smgrnblocks(). But I'll take it into consideration.

v29-0003-Optimize-DropRelFileNodeBuffers-during-recovery
----------------------------------------------------------------------
------------
2.
+ /* Check that it is in the buffer pool. If not, do nothing. */
+ LWLockAcquire(bufPartitionLock, LW_SHARED); buf_id =
+ BufTableLookup(&bufTag, bufHash); LWLockRelease(bufPartitionLock);
+
+ if (buf_id < 0)
+ continue;
+
+ bufHdr = GetBufferDescriptor(buf_id);
+
+ buf_state = LockBufHdr(bufHdr);
+
+ if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&

I think a pre-check for RelFileNode might be better before LockBufHdr
for the reasons mentioned in this function few lines down.

The equivalent check is already done by BufTableLookup(). The last line in
the above is not a precheck but the final check.

Which check in that API you are talking about? Are you telling because
we are trying to use a hash value corresponding to rnode.node to find
the block then I don't think it is equivalent because there is a
difference in actual values. But even if we want to rely on that, a
comment is required but I guess we can do the check as well because it
shouldn't be a costly pre-check.

4.
+ /*
+ * Look up the buffer in the hashtable if the block size is known to
+ * be accurate and the total blocks to be invalidated is below the
+ * full scan threshold.  Otherwise, give up the optimization.
+ */
+ if (accurate && nBlocksToInvalidate <

BUF_DROP_FULL_SCAN_THRESHOLD)

+ { for (j = 0; j < nforks; j++) { BlockNumber curBlock;
+
+ for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j];
+ curBlock++) {
+ uint32 bufHash; /* hash value for tag */ BufferTag bufTag; /*
+ identity of requested block */
+ LWLock     *bufPartitionLock; /* buffer partition lock for it */
+ int buf_id;
+
+ /* create a tag so we can lookup the buffer */
+ INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+ /* determine its hash code and partition lock ID */ bufHash =
+ BufTableHashCode(&bufTag); bufPartitionLock =
+ BufMappingPartitionLock(bufHash);
+
+ /* Check that it is in the buffer pool. If not, do nothing. */
+ LWLockAcquire(bufPartitionLock, LW_SHARED); buf_id =
+ BufTableLookup(&bufTag, bufHash); LWLockRelease(bufPartitionLock);
+
+ if (buf_id < 0)
+ continue;
+
+ bufHdr = GetBufferDescriptor(buf_id);
+
+ buf_state = LockBufHdr(bufHdr);
+
+ if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+ bufHdr->tag.forkNum == forkNum[j] && tag.blockNum >=
+ bufHdr->firstDelBlock[j])
+ InvalidateBuffer(bufHdr); /* releases spinlock */ else
+ UnlockBufHdr(bufHdr, buf_state); } } return; }

Can we move the code under this 'if' condition to a separate function,
say FindAndDropRelFileNodeBuffers or something like that?

Thinking about the TRUNCATE optimization, it sounds reasonable to have a
separate function, which runs the optmized dropping unconditionally.

Hmm, sure., although only DropRelFileNodeBuffers() would call the new function.
I guess it won't be a problem.

That shouldn't be a problem, you can make it a static function. It is
more from the code-readability perspective.

v29-0004-TRUNCATE-optimization
------------------------------------------------
5.
+ for (i = 0; i < n; i++)
+ {
+ nforks = 0;
+ nBlocksToInvalidate = 0;
+
+ for (j = 0; j <= MAX_FORKNUM; j++)
+ {
+ if (!smgrexists(rels[i], j))
+ continue;
+
+ /* Get the number of blocks for a relation's fork */ nblocks =
+ smgrnblocks(rels[i], j, NULL);
+
+ nBlocksToInvalidate += nblocks;
+
+ forks[nforks++] = j;
+ }
+ if (nBlocksToInvalidate >= BUF_DROP_FULL_SCAN_THRESHOLD)

goto

+ buffer_full_scan;
+
+ DropRelFileNodeBuffers(rels[i], forks, nforks, firstDelBlocks); }
+ pfree(nodes); pfree(rels); pfree(rnodes); return;

I think this can be slower than the current Truncate. Say there are
BUF_DROP_FULL_SCAN_THRESHOLD then you would anyway have to

scan the

entire shared buffers so the work done in optimized path for other two
relations will add some over head.

That's true. The criteria here is the number of blocks of all relations. And
even if all of the relations is smaller than the threshold, we should go to the
full-scan dropping if the total size exceeds the threshold. So we cannot
reuse DropRelFileNodeBuffers() as is here.

Also, as written, I think you need to remove the nodes for which you
have invalidated the buffers via optimized path, no.

Right, in the current patch it is indeed slower.
But the decision criteria whether to optimize or not is decided per relation,
not for all relations. So there is a possibility that we have already invalidated
buffers of the first relation, but the next relation buffers exceed the threshold that we
need to do the full scan. So yes that should be fixed. Remove the nodes that we
have already invalidated so that we don't scan them anymore when scanning NBuffers.
I will fix in the next version.

Thank you for the helpful feedback. I'll upload the updated set of patches soon
also when we reach a consensus on the boolean parameter name too.

Sure, but feel free to leave the truncate optimization patch for now,
we can do that as a follow-up patch once the vacuum-optimization patch
is committed. Horiguchi-San, are you fine with this approach?

--
With Regards,
Amit Kapila.

#148Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#147)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 5 Nov 2020 11:07:21 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Thu, Nov 5, 2020 at 8:26 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Few comments on patches:
======================
v29-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks
----------------------------------------------------------------------
-------------
1.
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
{
BlockNumber result;
/*
* For now, we only use cached values in recovery due to lack of a
shared
- * invalidation mechanism for changes in file size.
+ * invalidation mechanism for changes in file size.  The cached
+ values
+ * could be smaller than the actual number of existing buffers of the file.
+ * This is caused by lseek of buggy Linux kernels that might not have
+ * accounted for the recent write.
*/
if (InRecovery && reln->smgr_cached_nblocks[forknum] !=
InvalidBlockNumber)
+ {
+ if (accurate != NULL)
+ *accurate = true;
+

I don't understand this comment. Few emails back, I think we have
discussed that cached value can't be less than the number of buffers
during recovery. If that happens to be true then we have some problem.
If you want to explain 'accurate' variable then you can do the same
atop of function. Would it be better to name this variable as
'cached'?

(I agree that the comment needs to be fixed.)

FWIW I don't think 'cached' suggests the characteristics of the returned value
on its interface. It was introduced to reduce fseek() calls, and after that we
have found that it can be regarded as the authoritative source of the file size.
The "accurate" means that it is guaranteed that we don't have a buffer for the
file blocks further than that number. I don't come up with a more proper
word than "accurate" but also I don't think "cached" is proper here.

Sure but that is not the guarantee this API gives. It has to be
guaranteed by the logic else-where, so not sure if it is a good idea
to try to reflect the same here. The comments in the caller where we
use this should explain why it is safe to use this value.

Isn't it already guaranteed by the bugmgr code that we don't have
buffers for nonexistent file blocks? What is needed here is, yeah,
the returned value from smgrblocks is "reliable". If "reliable" is
still not proper, I give up and agree to "cached".

I also couldn't think of a better parameter name. Accurate seems to be better fit
as it describes a measurement close to an accepted value.
How about fixing the comment like below, would this suffice?

/*
* smgrnblocks() -- Calculate the number of blocks in the
* supplied relation.
*
* accurate flag acts as an authoritative source of the file size and
* ensures that no buffers exist for blocks after the file size is known
* to the process.
*/
BlockNumber
smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
{
BlockNumber result;

/*
* For now, we only use cached values in recovery due to lack of a shared
* invalidation mechanism for changes in file size. In recovery, the cached
* value returned by the first lseek could be smaller than the actual number
* of existing buffers of the file, which is caused by buggy Linux kernels
* that might not have accounted for the recent write. However, we can
* still rely on the cached value even if we get a bogus value from first
* lseek since it is impossible to have buffer for blocks after the file size.
*/

By the way, if there's a case where we extend a file by more than one block the
cached value becomes invalid. I'm not sure if it actually happens, but the
following sequence may lead to a problem. We need a protection for that
case.

smgrnblocks() : cached n
truncate to n-5 : cached n=5
extend to m + 2 : cached invalid
(fsync failed)
smgrnblocks() : returns and cached n-5

I think one possible idea is to actually commit the Assert patch
(v29-0001-Prevent-invalidating-blocks-in-smgrextend-during) to ensure
that it can't happen during recovery. And even if it happens why would
there be any buffer with the block in it left when the fsync failed?
And if there is no buffer with a block which doesn't account due to
lseek lies then there shouldn't be any problem. Do you have any other
ideas on what better can be done here?

Ouch! Sorry for the confusion. I confused that patch touches the
truncation side. Yes the 0001 does that.

I am not sure if the patch should cover this or should be a separate thread altogether since
a number of functions also rely on the smgrnblocks(). But I'll take it into consideration.

v29-0003-Optimize-DropRelFileNodeBuffers-during-recovery
----------------------------------------------------------------------
------------
2.
+ /* Check that it is in the buffer pool. If not, do nothing. */
+ LWLockAcquire(bufPartitionLock, LW_SHARED); buf_id =
+ BufTableLookup(&bufTag, bufHash); LWLockRelease(bufPartitionLock);
+
+ if (buf_id < 0)
+ continue;
+
+ bufHdr = GetBufferDescriptor(buf_id);
+
+ buf_state = LockBufHdr(bufHdr);
+
+ if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&

I think a pre-check for RelFileNode might be better before LockBufHdr
for the reasons mentioned in this function few lines down.

The equivalent check is already done by BufTableLookup(). The last line in
the above is not a precheck but the final check.

Which check in that API you are talking about? Are you telling because
we are trying to use a hash value corresponding to rnode.node to find
the block then I don't think it is equivalent because there is a
difference in actual values. But even if we want to rely on that, a
comment is required but I guess we can do the check as well because it
shouldn't be a costly pre-check.

I think the only problematic case is that BufTableLookup wrongly
misses buffers actually to be dropped. (And the case of too-many
false-positives, not critical though.) If omission is the case, we
cannot adopt this optimization at all. And if the false-positive is
the case, maybe we need to adopt more restrictive prechecking, but
RelFileNodeEquals is *not* more restrictive than BufTableLookup in the
first place.

What case do you think is problematic when considering
BufTableLookup() as the prechecking?

4.
+ /*
+ * Look up the buffer in the hashtable if the block size is known to
+ * be accurate and the total blocks to be invalidated is below the
+ * full scan threshold.  Otherwise, give up the optimization.
+ */
+ if (accurate && nBlocksToInvalidate <

BUF_DROP_FULL_SCAN_THRESHOLD)

+ { for (j = 0; j < nforks; j++) { BlockNumber curBlock;
+
+ for (curBlock = firstDelBlock[j]; curBlock < nForkBlocks[j];
+ curBlock++) {
+ uint32 bufHash; /* hash value for tag */ BufferTag bufTag; /*
+ identity of requested block */
+ LWLock     *bufPartitionLock; /* buffer partition lock for it */
+ int buf_id;
+
+ /* create a tag so we can lookup the buffer */
+ INIT_BUFFERTAG(bufTag, rnode.node, forkNum[j], curBlock);
+
+ /* determine its hash code and partition lock ID */ bufHash =
+ BufTableHashCode(&bufTag); bufPartitionLock =
+ BufMappingPartitionLock(bufHash);
+
+ /* Check that it is in the buffer pool. If not, do nothing. */
+ LWLockAcquire(bufPartitionLock, LW_SHARED); buf_id =
+ BufTableLookup(&bufTag, bufHash); LWLockRelease(bufPartitionLock);
+
+ if (buf_id < 0)
+ continue;
+
+ bufHdr = GetBufferDescriptor(buf_id);
+
+ buf_state = LockBufHdr(bufHdr);
+
+ if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
+ bufHdr->tag.forkNum == forkNum[j] && tag.blockNum >=
+ bufHdr->firstDelBlock[j])
+ InvalidateBuffer(bufHdr); /* releases spinlock */ else
+ UnlockBufHdr(bufHdr, buf_state); } } return; }

Can we move the code under this 'if' condition to a separate function,
say FindAndDropRelFileNodeBuffers or something like that?

Thinking about the TRUNCATE optimization, it sounds reasonable to have a
separate function, which runs the optmized dropping unconditionally.

Hmm, sure., although only DropRelFileNodeBuffers() would call the new function.
I guess it won't be a problem.

That shouldn't be a problem, you can make it a static function. It is
more from the code-readability perspective.

Sure, but feel free to leave the truncate optimization patch for now,
we can do that as a follow-up patch once the vacuum-optimization patch
is committed. Horiguchi-San, are you fine with this approach?

Of course. I don't think we have to commit the two at once at all.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#149Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#148)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Nov 5, 2020 at 1:59 PM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Thu, 5 Nov 2020 11:07:21 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Thu, Nov 5, 2020 at 8:26 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Few comments on patches:
======================
v29-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks
----------------------------------------------------------------------
-------------
1.
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *accurate)
{
BlockNumber result;
/*
* For now, we only use cached values in recovery due to lack of a
shared
- * invalidation mechanism for changes in file size.
+ * invalidation mechanism for changes in file size.  The cached
+ values
+ * could be smaller than the actual number of existing buffers of the file.
+ * This is caused by lseek of buggy Linux kernels that might not have
+ * accounted for the recent write.
*/
if (InRecovery && reln->smgr_cached_nblocks[forknum] !=
InvalidBlockNumber)
+ {
+ if (accurate != NULL)
+ *accurate = true;
+

I don't understand this comment. Few emails back, I think we have
discussed that cached value can't be less than the number of buffers
during recovery. If that happens to be true then we have some problem.
If you want to explain 'accurate' variable then you can do the same
atop of function. Would it be better to name this variable as
'cached'?

(I agree that the comment needs to be fixed.)

FWIW I don't think 'cached' suggests the characteristics of the returned value
on its interface. It was introduced to reduce fseek() calls, and after that we
have found that it can be regarded as the authoritative source of the file size.
The "accurate" means that it is guaranteed that we don't have a buffer for the
file blocks further than that number. I don't come up with a more proper
word than "accurate" but also I don't think "cached" is proper here.

Sure but that is not the guarantee this API gives. It has to be
guaranteed by the logic else-where, so not sure if it is a good idea
to try to reflect the same here. The comments in the caller where we
use this should explain why it is safe to use this value.

Isn't it already guaranteed by the bugmgr code that we don't have
buffers for nonexistent file blocks? What is needed here is, yeah,
the returned value from smgrblocks is "reliable". If "reliable" is
still not proper, I give up and agree to "cached".

I still feel 'cached' is a better name.

I am not sure if the patch should cover this or should be a separate thread altogether since
a number of functions also rely on the smgrnblocks(). But I'll take it into consideration.

v29-0003-Optimize-DropRelFileNodeBuffers-during-recovery
----------------------------------------------------------------------
------------
2.
+ /* Check that it is in the buffer pool. If not, do nothing. */
+ LWLockAcquire(bufPartitionLock, LW_SHARED); buf_id =
+ BufTableLookup(&bufTag, bufHash); LWLockRelease(bufPartitionLock);
+
+ if (buf_id < 0)
+ continue;
+
+ bufHdr = GetBufferDescriptor(buf_id);
+
+ buf_state = LockBufHdr(bufHdr);
+
+ if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&

I think a pre-check for RelFileNode might be better before LockBufHdr
for the reasons mentioned in this function few lines down.

The equivalent check is already done by BufTableLookup(). The last line in
the above is not a precheck but the final check.

Which check in that API you are talking about? Are you telling because
we are trying to use a hash value corresponding to rnode.node to find
the block then I don't think it is equivalent because there is a
difference in actual values. But even if we want to rely on that, a
comment is required but I guess we can do the check as well because it
shouldn't be a costly pre-check.

I think the only problematic case is that BufTableLookup wrongly
misses buffers actually to be dropped. (And the case of too-many
false-positives, not critical though.) If omission is the case, we
cannot adopt this optimization at all. And if the false-positive is
the case, maybe we need to adopt more restrictive prechecking, but
RelFileNodeEquals is *not* more restrictive than BufTableLookup in the
first place.

What case do you think is problematic when considering
BufTableLookup() as the prechecking?

I was slightly worried about false-positives but on again thinking
about it, I think we don't need any additional pre-check here.

--
With Regards,
Amit Kapila.

#150Thomas Munro
thomas.munro@gmail.com
In reply to: Amit Kapila (#149)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Nov 5, 2020 at 10:47 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

I still feel 'cached' is a better name.

Amusingly, this thread is hitting all the hardest problems in computer
science according to the well known aphorism...

Here's a devil's advocate position I thought about: It's OK to leave
stray buffers (clean or dirty) in the buffer pool if files are
truncated underneath us by gremlins, as long as your system eventually
crashes before completing a checkpoint. The OID can't be recycled
until after a successful checkpoint, so the stray blocks can't be
confused with the blocks of another relation, and weird errors are
expected on a system that is in serious trouble. It's actually much
worse that we can give incorrect answers to queries when files are
truncated by gremlins (in the window of time before we presumably
crash because of EIO), because we're violating basic ACID principles
in user-visible ways. In this thread, discussion has focused on
availability (ie avoiding failures when trying to write back stray
buffers to a file that has been unlinked), but really a system that
can't see arbitrary committed transactions *shouldn't be available*.
This argument applies whether you think SEEK_END can only give weird
answers in the specific scenario I demonstrated with NFS, or whether
you think it's arbitrarily b0rked and reports random numbers: we
fundamentally can't tolerate that, so why are we trying to?

#151k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Kyotaro Horiguchi (#131)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, October 22, 2020 3:15 PM, Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote:

I'm not sure about the exact steps of the test, but it can be expected if we
have many small relations to truncate.

Currently BUF_DROP_FULL_SCAN_THRESHOLD is set to Nbuffers / 512,
which is quite arbitrary that comes from a wild guess.

Perhaps we need to run benchmarks that drops one relation with several
different ratios between the number of buffers to-be-dropped and Nbuffers,
and preferably both on spinning rust and SSD.

Sorry to get back to you on this just now.
Since we're prioritizing the vacuum patch, we also need to finalize which threshold value to use.
I proceeded testing with my latest set of patches because Amit-san's comments on the code, the ones we addressed,
don't really affect the performance. I'll post the updated patches for 0002 & 0003 after we come up with the proper
boolean parameter name for smgrnblocks and the buffer full scan threshold value.

Test the VACUUM performance with the following thresholds:
NBuffers/512, NBuffers/256, NBuffers/128,
and determine which of the ratio has the best performance in terms of speed.

I tested this on my machine (CPU 4v, 8GB memory, ext4) running on SSD.
Configure streaming replication environment.
shared_buffers = 100GB
autovacuum = off
full_page_writes = off
checkpoint_timeout = 30min

[Steps]
1. Create TABLE
2. INSERT data
3. DELETE from TABLE
4. Pause WAL replay on standby
5. VACUUM. Stop the primary.
6. Resume WAL replay and promote standby.

With 1 relation, there were no significant changes that we can observe:
(In seconds)
| s_b | Master | NBuffers/512 | NBuffers/256 | NBuffers/128 |
|-------|--------|--------------|--------------|--------------|
| 128MB | 0.106 | 0.105 | 0.105 | 0.105 |
| 100GB | 0.106 | 0.105 | 0.105 | 0.105 |

So I tested with 100 tables and got more convincing measurements:

| s_b | Master | NBuffers/512 | NBuffers/256 | NBuffers/128 |
|-------|--------|--------------|--------------|--------------|
| 128MB | 1.006 | 1.007 | 1.006 | 0.107 |
| 1GB | 0.706 | 0.606 | 0.606 | 0.605 |
| 20GB | 1.907 | 0.606 | 0.606 | 0.605 |
| 100GB | 7.013 | 0.706 | 0.606 | 0.607 |

The threshold NBuffers/128 has the best performance for default shared_buffers (128MB)
with 0.107 s, and equally performing with large shared_buffers up to 100GB.

We can use NBuffers/128 for the threshold, although I don't have a measurement for HDD yet.
However, I wonder if the above method would suffice to determine the final threshold that we
can use. If anyone has suggestions on how we can come up with the final value, like if I need
to modify some steps above, I'd appreciate it.

Regarding the parameter name. Instead of accurate, we can use "cached" as originally intended from
the early versions of the patch since it is the smgr that handles smgrnblocks to get the the block
size of smgr_cached_nblocks.. "accurate" may confuse us because the cached value may not
be actually accurate..

Regards,
Kirk Jamison

#152Amit Kapila
amit.kapila16@gmail.com
In reply to: Thomas Munro (#150)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Nov 6, 2020 at 5:02 AM Thomas Munro <thomas.munro@gmail.com> wrote:

On Thu, Nov 5, 2020 at 10:47 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

I still feel 'cached' is a better name.

Amusingly, this thread is hitting all the hardest problems in computer
science according to the well known aphorism...

Here's a devil's advocate position I thought about: It's OK to leave
stray buffers (clean or dirty) in the buffer pool if files are
truncated underneath us by gremlins, as long as your system eventually
crashes before completing a checkpoint. The OID can't be recycled
until after a successful checkpoint, so the stray blocks can't be
confused with the blocks of another relation, and weird errors are
expected on a system that is in serious trouble. It's actually much
worse that we can give incorrect answers to queries when files are
truncated by gremlins (in the window of time before we presumably
crash because of EIO), because we're violating basic ACID principles
in user-visible ways. In this thread, discussion has focused on
availability (ie avoiding failures when trying to write back stray
buffers to a file that has been unlinked), but really a system that
can't see arbitrary committed transactions *shouldn't be available*.
This argument applies whether you think SEEK_END can only give weird
answers in the specific scenario I demonstrated with NFS, or whether
you think it's arbitrarily b0rked and reports random numbers: we
fundamentally can't tolerate that, so why are we trying to?

It is not very clear to me how this argument applies to the patch
in-discussion where we are relying on the cached value of blocks
during recovery. I understand your point that we might skip scanning
the pages and thus might not show some recently added data but that
point is not linked with what we are trying to do with this patch.
AFAIU, the theory we discussed above is that there shouldn't be any
stray blocks in the buffers with this patch because even if the
smgrnblocks(SEEK_END) didn't gave us the right answers, we shouldn't
have any buffers for the blocks after the size returned by smgrnblocks
during recovery. I think the problem could happen if we extend the
relation by multiple blocks which will invalidate the cached value
during recovery and then probably the future calls to smgrnblocks can
lead to problems if it lies with us but as far as I know we don't do
that. Can you please be more specific how this patch can lead to a
problem?

--
With Regards,
Amit Kapila.

#153Thomas Munro
thomas.munro@gmail.com
In reply to: Amit Kapila (#152)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Nov 6, 2020 at 5:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Fri, Nov 6, 2020 at 5:02 AM Thomas Munro <thomas.munro@gmail.com> wrote:

Here's a devil's advocate position I thought about: It's OK to leave
stray buffers (clean or dirty) in the buffer pool if files are
truncated underneath us by gremlins, as long as your system eventually
crashes before completing a checkpoint. The OID can't be recycled
until after a successful checkpoint, so the stray blocks can't be
confused with the blocks of another relation, and weird errors are
expected on a system that is in serious trouble. It's actually much
worse that we can give incorrect answers to queries when files are
truncated by gremlins (in the window of time before we presumably
crash because of EIO), because we're violating basic ACID principles
in user-visible ways. In this thread, discussion has focused on
availability (ie avoiding failures when trying to write back stray
buffers to a file that has been unlinked), but really a system that
can't see arbitrary committed transactions *shouldn't be available*.
This argument applies whether you think SEEK_END can only give weird
answers in the specific scenario I demonstrated with NFS, or whether
you think it's arbitrarily b0rked and reports random numbers: we
fundamentally can't tolerate that, so why are we trying to?

It is not very clear to me how this argument applies to the patch
in-discussion where we are relying on the cached value of blocks
during recovery. I understand your point that we might skip scanning
the pages and thus might not show some recently added data but that
point is not linked with what we are trying to do with this patch.

It's an argument for giving up the hard-to-name cache trick completely
and going back to using unmodified smgrnblocks(), both in recovery and
online. If the only mechanism for unexpected file shrinkage is
writeback failure, then your system will be panicking soon enough
anyway -- so is it really that bad if there are potentially some other
weird errors logged some time before that? Maybe those errors will
even take the system down sooner, and maybe that's appropriate? If
there are other mechanisms for random file shrinkage that don't imply
a panic in your near future, then we have bigger problems that can't
be solved by any number of bandaids, at least not without
understanding the details of this hypothetical unknown failure mode.

The main argument I can think of against the idea of using plain old
smgrnblocks() is that the current error messages on smgrwrite()
failure for stray blocks would be indistinguishible from cases where
an external actor unlinked the file. I don't mind getting an error
that prevents checkpointing -- your system is in big trouble! -- but
it'd be nice to be able to detect that *we* unlinked the file,
implying the filesystem and bufferpool are out of sync, and spit out a
special diagnostic message. I suppose if it's the checkpointer doing
the writing, it could check if the relfilenode is on the
queued-up-for-delete-after-the-checkpoint list, and if so, it could
produce a different error message just for this edge case.
Unfortunately that's not a general solution, because any backend might
try to write a buffer out and they aren't synchronised with
checkpoints.

I'm not sure what the best approach is. It'd certainly be nice to be
able to drop small tables quickly online too, as a benefit of this
approach.

#154Amit Kapila
amit.kapila16@gmail.com
In reply to: Thomas Munro (#153)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Nov 6, 2020 at 11:10 AM Thomas Munro <thomas.munro@gmail.com> wrote:

On Fri, Nov 6, 2020 at 5:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

It is not very clear to me how this argument applies to the patch
in-discussion where we are relying on the cached value of blocks
during recovery. I understand your point that we might skip scanning
the pages and thus might not show some recently added data but that
point is not linked with what we are trying to do with this patch.

It's an argument for giving up the hard-to-name cache trick completely
and going back to using unmodified smgrnblocks(), both in recovery and
online. If the only mechanism for unexpected file shrinkage is
writeback failure, then your system will be panicking soon enough
anyway

How else (except for writeback failure due to unexpected shrinkage)
the system will panic? Are you saying that if users don't get some
data due to lseek lying with us then it will be equivalent to panic or
are you indicating the scenario where ReadBuffer_common gives error
"unexpected data beyond EOF ...."?

-- so is it really that bad if there are potentially some other
weird errors logged some time before that? Maybe those errors will
even take the system down sooner, and maybe that's appropriate?

Yeah, it might be appropriate to panic in such situations but
ReadBuffer_common gives an error and ask user to update the system.

If
there are other mechanisms for random file shrinkage that don't imply
a panic in your near future, then we have bigger problems that can't
be solved by any number of bandaids, at least not without
understanding the details of this hypothetical unknown failure mode.

I think one of the problems is returning fewer rows and that too
without any warning or error, so maybe that is a bigger problem but we
seem to be okay with it as that is already a known thing though I
think that is not documented anywhere.

The main argument I can think of against the idea of using plain old
smgrnblocks() is that the current error messages on smgrwrite()
failure for stray blocks would be indistinguishible from cases where
an external actor unlinked the file. I don't mind getting an error
that prevents checkpointing -- your system is in big trouble! -- but
it'd be nice to be able to detect that *we* unlinked the file,
implying the filesystem and bufferpool are out of sync, and spit out a
special diagnostic message. I suppose if it's the checkpointer doing
the writing, it could check if the relfilenode is on the
queued-up-for-delete-after-the-checkpoint list, and if so, it could
produce a different error message just for this edge case.
Unfortunately that's not a general solution, because any backend might
try to write a buffer out and they aren't synchronised with
checkpoints.

Yeah, but I am not sure if we can consider manual (external actor)
tinkering with the files the same as something that happened due to
the database server relying on the wrong information.

I'm not sure what the best approach is. It'd certainly be nice to be
able to drop small tables quickly online too, as a benefit of this
approach.

Right, that is why I was thinking to do it only for recovery where it
is safe from the database server perspective. OTOH, if we broadly
accept that any time filesystem lies with us the behavior could be
unpredictable like the system can return fewer rows than expected or
it could cause panic. I think there is an argument that it might be
better to error out (even with panic) rather than silently returning
fewer rows but unfortunately detecting the same in each and every case
doesn't seem feasible.

One vague idea could be to develop pg_test_seek which can detect such
problems but not sure if we can rely on such a tool to always give us
the right answer. Were you able to consistently reproduce the lseek
problem on the system where you have tried?

--
With Regards,
Amit Kapila.

#155k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#151)
4 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: k.jamison@fujitsu.com <k.jamison@fujitsu.com>
On Thursday, October 22, 2020 3:15 PM, Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

I'm not sure about the exact steps of the test, but it can be expected
if we have many small relations to truncate.

Currently BUF_DROP_FULL_SCAN_THRESHOLD is set to Nbuffers / 512,

which

is quite arbitrary that comes from a wild guess.

Perhaps we need to run benchmarks that drops one relation with several
different ratios between the number of buffers to-be-dropped and
Nbuffers, and preferably both on spinning rust and SSD.

Sorry to get back to you on this just now.
Since we're prioritizing the vacuum patch, we also need to finalize which
threshold value to use.
I proceeded testing with my latest set of patches because Amit-san's
comments on the code, the ones we addressed, don't really affect the
performance. I'll post the updated patches for 0002 & 0003 after we come up
with the proper boolean parameter name for smgrnblocks and the buffer full
scan threshold value.

Test the VACUUM performance with the following thresholds:
NBuffers/512, NBuffers/256, NBuffers/128, and determine which of the
ratio has the best performance in terms of speed.

I tested this on my machine (CPU 4v, 8GB memory, ext4) running on SSD.
Configure streaming replication environment.
shared_buffers = 100GB
autovacuum = off
full_page_writes = off
checkpoint_timeout = 30min

[Steps]
1. Create TABLE
2. INSERT data
3. DELETE from TABLE
4. Pause WAL replay on standby
5. VACUUM. Stop the primary.
6. Resume WAL replay and promote standby.

With 1 relation, there were no significant changes that we can observe:
(In seconds)
| s_b | Master | NBuffers/512 | NBuffers/256 | NBuffers/128 |
|-------|--------|--------------|--------------|--------------|
| 128MB | 0.106 | 0.105 | 0.105 | 0.105 |
| 100GB | 0.106 | 0.105 | 0.105 | 0.105 |

So I tested with 100 tables and got more convincing measurements:

| s_b | Master | NBuffers/512 | NBuffers/256 | NBuffers/128 |
|-------|--------|--------------|--------------|--------------|
| 128MB | 1.006 | 1.007 | 1.006 | 0.107 |
| 1GB | 0.706 | 0.606 | 0.606 | 0.605 |
| 20GB | 1.907 | 0.606 | 0.606 | 0.605 |
| 100GB | 7.013 | 0.706 | 0.606 | 0.607 |

The threshold NBuffers/128 has the best performance for default
shared_buffers (128MB) with 0.107 s, and equally performing with large
shared_buffers up to 100GB.

We can use NBuffers/128 for the threshold, although I don't have a
measurement for HDD yet.
However, I wonder if the above method would suffice to determine the final
threshold that we can use. If anyone has suggestions on how we can come
up with the final value, like if I need to modify some steps above, I'd
appreciate it.

Regarding the parameter name. Instead of accurate, we can use "cached" as
originally intended from the early versions of the patch since it is the smgr
that handles smgrnblocks to get the the block size of smgr_cached_nblocks..
"accurate" may confuse us because the cached value may not be actually
accurate..

Hi,

So I proceeded to update the patches using the "cached" parameter and updated
the corresponding comments to it in 0002.

I've addressed the suggestions and comments of Amit-san on 0003:
1. For readability, I moved the code block to a new static function FindAndDropRelFileNodeBuffers()
2. Initialize the bool cached with false.
3. It's also decided that we don't need the extra pre-checking of RelFileNode
when locking the bufhdr in FindAndDropRelFileNodeBuffers

I repeated the recovery performance test for vacuum. (I made a mistake previously in NBuffers/128)
The 3 kinds of thresholds are almost equally performant. I chose NBuffers/256 for this patch.

| s_b | Master | NBuffers/512 | NBuffers/256 | NBuffers/128 |
|-------|--------|--------------|--------------|--------------|
| 128MB | 1.006 | 1.007 | 1.007 | 1.007 |
| 1GB | 0.706 | 0.606 | 0.606 | 0.606 |
| 20GB | 1.907 | 0.606 | 0.606 | 0.606 |
| 100GB | 7.013 | 0.706 | 0.606 | 0.606 |

Although we said that we'll prioritize vacuum optimization first, I've also updated the 0004 patch
(truncate optimization) which addresses the previous concern of slower truncate due to
redundant lookup of already-dropped buffers. In the new patch, we initially drop relation buffers
using the optimized DropRelFileNodeBuffers() if the buffers do not exceed the full-scan threshold,
then later on we drop the remaining buffers using full-scan.

Regards,
Kirk Jamison

Attachments:

v30-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v30-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From e38e71ca8cad21a6591fefe463565a59082272cb Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v30 1/4] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v30-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v30-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From 56de88968da9252c5bccf64857b738510e84a557 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v30 2/4] Add bool param in smgrnblocks() for cached blocks.

The flag ensures that we return a reliable value from smgrnblocks.
---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 21 ++++++++++++++++++---
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 34 insertions(+), 18 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 7e915bc..46fd97d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index d538f257..3874ff3 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -721,7 +721,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ad0d1a9..1680bf4 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -740,7 +740,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2857,7 +2857,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..e9dffd2 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -535,7 +535,6 @@ smgrwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 										buffer, skipFsync);
 }
 
-
 /*
  *	smgrwriteback() -- Trigger kernel writeback for the supplied range of
  *					   blocks.
@@ -551,18 +550,34 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 /*
  *	smgrnblocks() -- Calculate the number of blocks in the
  *					 supplied relation.
+ *
+ *		The "cached" flag ensures that no buffers exist for blocks after the
+ *		cached value is known to the process during recovery.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *cached)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  In recovery, the cached
+	 * value returned by the first lseek could be smaller than the actual number
+	 * of existing buffers of the file, which is caused by buggy Linux kernels
+	 * that might not have accounted for the recent write.  However, we can still
+	 * rely on the cached value even if we get a bogus value from first lseek
+	 * since it is impossible to have buffer for blocks after that file size.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (cached != NULL)
+			*cached = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
+
+	if (cached != NULL)
+		*cached = false;
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..af11b53 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *accurate);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v30-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v30-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From b75d7e9502e532b6582081e960324ee5b82720f5 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Fri, 6 Nov 2020 07:14:20 +0000
Subject: [PATCH v30 3/4] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning. This improves the
performance especially when VACUUM or autovacuum truncated off any
of the empty pages at the end of relation.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 115 +++++++++++++++++++++++++++++++++---
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 110 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 1680bf4..fa557d7 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 256)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,11 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FindAndDropRelFileNodeBuffers(RelFileNode rnode,
+										  ForkNumber *forkNum,
+										  int nforks,
+										  BlockNumber *nForkBlocks,
+										  BlockNumber *firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2967,18 +2974,25 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are already cached. See smgrnblocks() for more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		cached = false;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2992,6 +3006,35 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of to-be-invalidated blocks of a relation as
+	 * well as the total blocks for a given fork.  Give up the optimization
+	 * if the block is not cached.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &cached);
+
+		if (!cached)
+			break;
+
+		/* Get the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffers in the hashtable and drop them if the block size
+	 * is already cached and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		FindAndDropRelFileNodeBuffers(rnode.node, forkNum, nforks,
+									  nForkBlocks, firstDelBlock);
+		return;
+	}
+
 	for (i = 0; i < NBuffers; i++)
 	{
 		BufferDesc *bufHdr = GetBufferDescriptor(i);
@@ -3135,6 +3178,65 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 }
 
 /* ---------------------------------------------------------------------
+ *		FindAndDropRelFileNodeBuffers
+ *
+ *		This function finds and removes from the buffer pool all the pages
+ *		of the specified relation forks that have block numbers >= firstDelBlock.
+ *		(In particular, with firstDelBlock = 0, all pages are removed.)
+ *		This is only called in recovery when the block count of any fork is
+ *		cached and the total number of to-be-invalidated blocks per relation
+ *		do not exceed the threshold for full buffer scan.
+ * --------------------------------------------------------------------
+ */
+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+							  BlockNumber *nForkBlocks, BlockNumber *firstDelBlock)
+{
+	int		i;
+
+	for (i = 0; i < nforks; i++)
+	{
+		BlockNumber		curBlock;
+
+		for (curBlock = firstDelBlock[i]; curBlock < nForkBlocks[i]; curBlock++)
+		{
+			uint32		bufHash;		/* hash value for tag */
+			BufferTag	bufTag;			/* identity of requested block */
+			LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+			int		buf_id;
+			BufferDesc	*bufHdr;
+			uint32		buf_state;
+
+			/* create a tag so we can lookup the buffer */
+			INIT_BUFFERTAG(bufTag, rnode, forkNum[i], curBlock);
+
+			/* determine its hash code and partition lock ID */
+			bufHash = BufTableHashCode(&bufTag);
+			bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+			/* Check that it is in the buffer pool. If not, do nothing. */
+			LWLockAcquire(bufPartitionLock, LW_SHARED);
+			buf_id = BufTableLookup(&bufTag, bufHash);
+			LWLockRelease(bufPartitionLock);
+
+			if (buf_id < 0)
+				continue;
+
+			bufHdr = GetBufferDescriptor(buf_id);
+
+			buf_state = LockBufHdr(bufHdr);
+
+			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+				bufHdr->tag.forkNum == forkNum[i] &&
+				bufHdr->tag.blockNum >= firstDelBlock[i])
+				InvalidateBuffer(bufHdr);	/* releases spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);
+		}
+	}
+}
+
+/* ---------------------------------------------------------------------
  *		DropDatabaseBuffers
  *
  *		This function removes all the buffers in the buffer cache for a
@@ -3246,8 +3348,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index e9dffd2..9d3a67c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -605,7 +605,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v30-0004-TRUNCATE-optimization.patchapplication/octet-stream; name=v30-0004-TRUNCATE-optimization.patchDownload
From 964d9b8fc9acaf10552ba7a437b51e0abd304a5e Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Tue, 10 Nov 2020 02:43:13 +0000
Subject: [PATCH v30 4/4] TRUNCATE optimization.

DropRelFileNodesAllBuffers() is optimized to skip the time-consuming
scan of the whole buffer pool when the relation is small enough, or
when the number of blocks to be invalidated is below the full scan
threshold. This improves the performance when the TRUNCATE command
truncated off any of the empty pages at the end of relation.
---
 src/backend/storage/buffer/bufmgr.c | 90 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     | 14 +++---
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 89 insertions(+), 17 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index fa557d7..87862d9 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3086,17 +3086,34 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
+DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 {
 	int			i,
-				n = 0;
+				j,
+				nforks,
+				n = 0,
+				nrels = 0;
+	SMgrRelation	*rels;
+	RelFileNodeBackend *rnodes;
 	RelFileNode *nodes;
+	RelFileNode *undropped_nodes;
+	BlockNumber	nblocks;
+	BlockNumber	nBlocksToInvalidate;
+	BlockNumber	firstDelBlocks[MAX_FORKNUM + 1];
+	ForkNumber	forks[MAX_FORKNUM + 1];
 	bool		use_bsearch;
 
 	if (nnodes == 0)
 		return;
 
-	nodes = palloc(sizeof(RelFileNode) * nnodes);	/* non-local relations */
+	/* Create an array which contains all relations to be dropped */
+	rnodes = palloc(sizeof(RelFileNodeBackend) * nnodes);
+	for (i = 0; i < nnodes; i++)
+		rnodes[i] = smgr_reln[i]->smgr_rnode;
+
+	/* non-local relations */
+	rels = (SMgrRelation *)palloc(nnodes * sizeof(SMgrRelation));
+	nodes = palloc(sizeof(RelFileNode) * nnodes);
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	for (i = 0; i < nnodes; i++)
@@ -3107,7 +3124,10 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 				DropRelFileNodeAllLocalBuffers(rnodes[i].node);
 		}
 		else
+		{
+			rels[n] = smgr_reln[i];
 			nodes[n++] = rnodes[i].node;
+		}
 	}
 
 	/*
@@ -3117,6 +3137,56 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	if (n == 0)
 	{
 		pfree(nodes);
+		pfree(rels);
+		pfree(rnodes);
+		return;
+	}
+
+	/* undropped non-local relations */
+	undropped_nodes = palloc(sizeof(RelFileNode) * n);
+
+	/*
+	 * Zero the array of blocks because these will all be dropped anyway.
+	 * We don't care yet whether or not the block count of relation fork
+	 * is cached as it will be checked in DropRelFileNodeBuffers.  However,
+	 * we give up the optimization if the relation is large enough that the
+	 * number of blocks to be invalidated exceeds the threshold for full scan.
+	 */
+	memset(firstDelBlocks, 0, sizeof(firstDelBlocks));
+	for (i = 0; i < n; i++)
+	{
+		nforks = 0;
+		nBlocksToInvalidate = 0;
+
+		for (j = 0; j <= MAX_FORKNUM; j++)
+		{
+			if (!smgrexists(rels[i], j))
+				continue;
+
+			/* Get the number of blocks for a relation's fork */
+			nblocks = smgrnblocks(rels[i], j, NULL);
+
+			nBlocksToInvalidate += nblocks;
+
+			forks[nforks++] = j;
+		}
+
+		if (nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+			DropRelFileNodeBuffers(rels[i], forks, nforks, firstDelBlocks);
+		else
+			undropped_nodes[nrels++] = nodes[i];
+	}
+
+	/*
+	 * If there are no undropped nodes, then we're done. Release the
+	 * memory and return.
+	 */
+	if (nrels == 0)
+	{
+		pfree(undropped_nodes);
+		pfree(nodes);
+		pfree(rels);
+		pfree(rnodes);
 		return;
 	}
 
@@ -3126,11 +3196,11 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	 * an exactly determined value, as it depends on many factors (CPU and RAM
 	 * speeds, amount of shared buffers etc.).
 	 */
-	use_bsearch = n > RELS_BSEARCH_THRESHOLD;
+	use_bsearch = nrels > RELS_BSEARCH_THRESHOLD;
 
 	/* sort the list of rnodes if necessary */
 	if (use_bsearch)
-		pg_qsort(nodes, n, sizeof(RelFileNode), rnode_comparator);
+		pg_qsort(undropped_nodes, nrels, sizeof(RelFileNode), rnode_comparator);
 
 	for (i = 0; i < NBuffers; i++)
 	{
@@ -3149,9 +3219,9 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 
 			for (j = 0; j < n; j++)
 			{
-				if (RelFileNodeEquals(bufHdr->tag.rnode, nodes[j]))
+				if (RelFileNodeEquals(bufHdr->tag.rnode, undropped_nodes[j]))
 				{
-					rnode = &nodes[j];
+					rnode = &undropped_nodes[j];
 					break;
 				}
 			}
@@ -3159,7 +3229,7 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 		else
 		{
 			rnode = bsearch((const void *) &(bufHdr->tag.rnode),
-							nodes, n, sizeof(RelFileNode),
+							undropped_nodes, nrels, sizeof(RelFileNode),
 							rnode_comparator);
 		}
 
@@ -3173,8 +3243,10 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 		else
 			UnlockBufHdr(bufHdr, buf_state);
 	}
-
+	pfree(undropped_nodes);
 	pfree(nodes);
+	pfree(rels);
+	pfree(rnodes);
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 9d3a67c..3663bb7 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -392,7 +392,13 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 		return;
 
 	/*
-	 * create an array which contains all relations to be dropped, and close
+	 * Get rid of any remaining buffers for the relations.  bufmgr will just
+	 * drop them without bothering to write the contents.
+	 */
+	DropRelFileNodesAllBuffers(rels, nrels);
+
+	/*
+	 * Create an array which contains all relations to be dropped, and close
 	 * each relation's forks at the smgr level while at it
 	 */
 	rnodes = palloc(sizeof(RelFileNodeBackend) * nrels);
@@ -409,12 +415,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	}
 
 	/*
-	 * Get rid of any remaining buffers for the relations.  bufmgr will just
-	 * drop them without bothering to write the contents.
-	 */
-	DropRelFileNodesAllBuffers(rnodes, nrels);
-
-	/*
 	 * It'd be nice to tell the stats collector to forget them immediately,
 	 * too. But we can't because we don't know the OIDs.
 	 */
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 056f65e..2e5189b 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -205,7 +205,7 @@ extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels)
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
-extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
+extern void DropRelFileNodesAllBuffers(struct SMgrRelationData **smgr_reln, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
 #define RelationGetNumberOfBlocks(reln) \
-- 
1.8.3.1

#156Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#155)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Nov 10, 2020 at 8:19 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

From: k.jamison@fujitsu.com <k.jamison@fujitsu.com>
On Thursday, October 22, 2020 3:15 PM, Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

I'm not sure about the exact steps of the test, but it can be expected
if we have many small relations to truncate.

Currently BUF_DROP_FULL_SCAN_THRESHOLD is set to Nbuffers / 512,

which

is quite arbitrary that comes from a wild guess.

Perhaps we need to run benchmarks that drops one relation with several
different ratios between the number of buffers to-be-dropped and
Nbuffers, and preferably both on spinning rust and SSD.

Sorry to get back to you on this just now.
Since we're prioritizing the vacuum patch, we also need to finalize which
threshold value to use.
I proceeded testing with my latest set of patches because Amit-san's
comments on the code, the ones we addressed, don't really affect the
performance. I'll post the updated patches for 0002 & 0003 after we come up
with the proper boolean parameter name for smgrnblocks and the buffer full
scan threshold value.

Test the VACUUM performance with the following thresholds:
NBuffers/512, NBuffers/256, NBuffers/128, and determine which of the
ratio has the best performance in terms of speed.

I tested this on my machine (CPU 4v, 8GB memory, ext4) running on SSD.
Configure streaming replication environment.
shared_buffers = 100GB
autovacuum = off
full_page_writes = off
checkpoint_timeout = 30min

[Steps]
1. Create TABLE
2. INSERT data
3. DELETE from TABLE
4. Pause WAL replay on standby
5. VACUUM. Stop the primary.
6. Resume WAL replay and promote standby.

With 1 relation, there were no significant changes that we can observe:
(In seconds)
| s_b | Master | NBuffers/512 | NBuffers/256 | NBuffers/128 |
|-------|--------|--------------|--------------|--------------|
| 128MB | 0.106 | 0.105 | 0.105 | 0.105 |
| 100GB | 0.106 | 0.105 | 0.105 | 0.105 |

So I tested with 100 tables and got more convincing measurements:

| s_b | Master | NBuffers/512 | NBuffers/256 | NBuffers/128 |
|-------|--------|--------------|--------------|--------------|
| 128MB | 1.006 | 1.007 | 1.006 | 0.107 |
| 1GB | 0.706 | 0.606 | 0.606 | 0.605 |
| 20GB | 1.907 | 0.606 | 0.606 | 0.605 |
| 100GB | 7.013 | 0.706 | 0.606 | 0.607 |

The threshold NBuffers/128 has the best performance for default
shared_buffers (128MB) with 0.107 s, and equally performing with large
shared_buffers up to 100GB.

We can use NBuffers/128 for the threshold, although I don't have a
measurement for HDD yet.
However, I wonder if the above method would suffice to determine the final
threshold that we can use. If anyone has suggestions on how we can come
up with the final value, like if I need to modify some steps above, I'd
appreciate it.

Regarding the parameter name. Instead of accurate, we can use "cached" as
originally intended from the early versions of the patch since it is the smgr
that handles smgrnblocks to get the the block size of smgr_cached_nblocks..
"accurate" may confuse us because the cached value may not be actually
accurate..

Hi,

So I proceeded to update the patches using the "cached" parameter and updated
the corresponding comments to it in 0002.

I've addressed the suggestions and comments of Amit-san on 0003:
1. For readability, I moved the code block to a new static function FindAndDropRelFileNodeBuffers()
2. Initialize the bool cached with false.
3. It's also decided that we don't need the extra pre-checking of RelFileNode
when locking the bufhdr in FindAndDropRelFileNodeBuffers

I repeated the recovery performance test for vacuum. (I made a mistake previously in NBuffers/128)
The 3 kinds of thresholds are almost equally performant. I chose NBuffers/256 for this patch.

| s_b | Master | NBuffers/512 | NBuffers/256 | NBuffers/128 |
|-------|--------|--------------|--------------|--------------|
| 128MB | 1.006 | 1.007 | 1.007 | 1.007 |
| 1GB | 0.706 | 0.606 | 0.606 | 0.606 |
| 20GB | 1.907 | 0.606 | 0.606 | 0.606 |
| 100GB | 7.013 | 0.706 | 0.606 | 0.606 |

I think this data is not very clear. What is the unit of time? What is
the size of the relation used for the test? Did the test use an
optimized path for all cases? If at 128MB, there is no performance
gain, can we consider the size of shared buffers as 256MB as well for
the threshold?

--
With Regards,
Amit Kapila.

#157Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#156)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Tue, 10 Nov 2020 08:33:26 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Tue, Nov 10, 2020 at 8:19 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

I repeated the recovery performance test for vacuum. (I made a mistake previously in NBuffers/128)
The 3 kinds of thresholds are almost equally performant. I chose NBuffers/256 for this patch.

| s_b | Master | NBuffers/512 | NBuffers/256 | NBuffers/128 |
|-------|--------|--------------|--------------|--------------|
| 128MB | 1.006 | 1.007 | 1.007 | 1.007 |
| 1GB | 0.706 | 0.606 | 0.606 | 0.606 |
| 20GB | 1.907 | 0.606 | 0.606 | 0.606 |
| 100GB | 7.013 | 0.706 | 0.606 | 0.606 |

I think this data is not very clear. What is the unit of time? What is
the size of the relation used for the test? Did the test use an
optimized path for all cases? If at 128MB, there is no performance
gain, can we consider the size of shared buffers as 256MB as well for
the threshold?

In the previous testing, it was shown as:

Recovery Time (in seconds)
| s_b | master | patched | %reg |
|-------|--------|---------|--------|
| 128MB | 3.043 | 2.977 | -2.22% |
| 1GB | 3.417 | 3.41 | -0.21% |
| 20GB | 20.597 | 2.409 | -755% |
| 100GB | 66.862 | 2.409 | -2676% |

So... The numbers seems to be in seconds, but the master gets about 10
times faster than this result for uncertain reasons. It seems that the
result contains something different from the difference by this patch
as a larger part.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#158Thomas Munro
thomas.munro@gmail.com
In reply to: Amit Kapila (#154)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Sat, Nov 7, 2020 at 12:40 AM Amit Kapila <amit.kapila16@gmail.com> wrote:

I think one of the problems is returning fewer rows and that too
without any warning or error, so maybe that is a bigger problem but we
seem to be okay with it as that is already a known thing though I
think that is not documented anywhere.

I'm not OK with it, and I'm not sure it's widely known or understood,
though I think we've made some progress in this thread. Perhaps, as a
separate project, we need to solve several related problems with a
shmem table of relation sizes from not-yet-synced files so that
smgrnblocks() is fast and always sees all preceding smgrextend()
calls. If we're going to need something like that anyway, and if we
can come up with a simple way to detect and report this type of
failure in the meantime, maybe this fast DROP project should just go
ahead and use the existing smgrnblocks() function without the weird
caching bandaid that only works in recovery?

The main argument I can think of against the idea of using plain old
smgrnblocks() is that the current error messages on smgrwrite()
failure for stray blocks would be indistinguishible from cases where
an external actor unlinked the file. I don't mind getting an error
that prevents checkpointing -- your system is in big trouble! -- but
it'd be nice to be able to detect that *we* unlinked the file,
implying the filesystem and bufferpool are out of sync, and spit out a
special diagnostic message. I suppose if it's the checkpointer doing
the writing, it could check if the relfilenode is on the
queued-up-for-delete-after-the-checkpoint list, and if so, it could
produce a different error message just for this edge case.
Unfortunately that's not a general solution, because any backend might
try to write a buffer out and they aren't synchronised with
checkpoints.

Yeah, but I am not sure if we can consider manual (external actor)
tinkering with the files the same as something that happened due to
the database server relying on the wrong information.

Here's a rough idea I thought of to detect this case; I'm not sure if
it has holes. When unlinking a relation, currently we truncate
segment 0 and unlink all the rest of the segments, and tell the
checkpointer to unlink segment 0 after the next checkpoint. What if
we also renamed segment 0 to "$X.dropped" (to be unlinked by the
checkpointer), and taught GetNewRelFileNode() to also skip anything
for which "$X.dropped" exists? Then mdwrite() could use
_mdfd_getseg(EXTENSION_RETURN_NULL), and if it gets NULL (= no file),
then it checks if "$X.dropped" exists, and if so it knows that it is
trying to write a stray block from a dropped relation in the buffer
pool. Then we panic, or warn but drop the write. The point of the
renaming is that (1) mdwrite() for segment 0 will detect the missing
file (not just higher segments), (2) every backends can see that a
relation has been recently dropped, while also interlocking with the
checkpointer though buffer locks.

One vague idea could be to develop pg_test_seek which can detect such
problems but not sure if we can rely on such a tool to always give us
the right answer. Were you able to consistently reproduce the lseek
problem on the system where you have tried?

Yeah, I can reproduce that reliably, but it requires quite a bit of
set-up as root so it might be tricky to package up in easy to run
form. It might be quite nice to prepare an easy-to-use "gallery of
weird buffered I/O effects" project, including some of the
local-filesystem-with-fault-injection stuff that Craig Ringer and
others were testing with a couple of years ago, but maybe not in the
pg repo.

#159k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Kyotaro Horiguchi (#157)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Tuesday, November 10, 2020 12:27 PM, Horiguchi-san wrote:

To: amit.kapila16@gmail.com
Cc: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>; Tsunakawa,
Takayuki/綱川 貴之 <tsunakawa.takay@fujitsu.com>; tgl@sss.pgh.pa.us;
andres@anarazel.de; robertmhaas@gmail.com;
tomas.vondra@2ndquadrant.com; pgsql-hackers@postgresql.org
Subject: Re: [Patch] Optimize dropping of relation buffers using dlist

At Tue, 10 Nov 2020 08:33:26 +0530, Amit Kapila <amit.kapila16@gmail.com>
wrote in

On Tue, Nov 10, 2020 at 8:19 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

I repeated the recovery performance test for vacuum. (I made a
mistake previously in NBuffers/128) The 3 kinds of thresholds are almost

equally performant. I chose NBuffers/256 for this patch.

| s_b | Master | NBuffers/512 | NBuffers/256 | NBuffers/128 |
|-------|--------|--------------|--------------|--------------|
| 128MB | 1.006 | 1.007 | 1.007 | 1.007 |
| 1GB | 0.706 | 0.606 | 0.606 | 0.606 |
| 20GB | 1.907 | 0.606 | 0.606 | 0.606 |
| 100GB | 7.013 | 0.706 | 0.606 | 0.606 |

I think this data is not very clear. What is the unit of time? What is
the size of the relation used for the test? Did the test use an
optimized path for all cases? If at 128MB, there is no performance
gain, can we consider the size of shared buffers as 256MB as well for
the threshold?

In the previous testing, it was shown as:

Recovery Time (in seconds)
| s_b | master | patched | %reg |
|-------|--------|---------|--------|
| 128MB | 3.043 | 2.977 | -2.22% |
| 1GB | 3.417 | 3.41 | -0.21% |
| 20GB | 20.597 | 2.409 | -755% |
| 100GB | 66.862 | 2.409 | -2676% |

So... The numbers seems to be in seconds, but the master gets about 10
times faster than this result for uncertain reasons. It seems that the result
contains something different from the difference by this patch as a larger
part.

The unit is in seconds.
The results that Horiguchi-san mentioned was the old test case I used where I vacuumed
database with 1000 relations that have been deleted.
I used a new test case in my last results that's why they're smaller:
VACUUM 1 parent table (350 MB) and 100 child partition tables (6 MB each)
in separate transcations after deleting the tables. After vacuum, the
parent table became 16kB and each child table was 2224kB.

I added the test for 256MB shared_buffers, and the performance is also almost the same.
We gain performance benefits for the larger shared_buffers.

| s_b | Master | NBuffers/512 | NBuffers/256 | NBuffers/128 |
|--------|--------|--------------|--------------|--------------|
| 128MB | 1.006 | 1.007 | 1.007 | 1.007 |
| 256 MB | 1.006 | 1.006 | 0.906 | 0.906 |
| 1GB | 0.706 | 0.606 | 0.606 | 0.606 |
| 20GB | 1.907 | 0.606 | 0.606 | 0.606 |
| 100GB | 7.013 | 0.706 | 0.606 | 0.606 |

Regards,
Kirk Jamison

#160Amit Kapila
amit.kapila16@gmail.com
In reply to: Thomas Munro (#158)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Nov 10, 2020 at 10:00 AM Thomas Munro <thomas.munro@gmail.com> wrote:

On Sat, Nov 7, 2020 at 12:40 AM Amit Kapila <amit.kapila16@gmail.com> wrote:

I think one of the problems is returning fewer rows and that too
without any warning or error, so maybe that is a bigger problem but we
seem to be okay with it as that is already a known thing though I
think that is not documented anywhere.

I'm not OK with it, and I'm not sure it's widely known or understood,

Yeah, it is quite possible but may be because we don't see many field
reports nobody thought of doing anything about it.

though I think we've made some progress in this thread. Perhaps, as a
separate project, we need to solve several related problems with a
shmem table of relation sizes from not-yet-synced files so that
smgrnblocks() is fast and always sees all preceding smgrextend()
calls. If we're going to need something like that anyway, and if we
can come up with a simple way to detect and report this type of
failure in the meantime, maybe this fast DROP project should just go
ahead and use the existing smgrnblocks() function without the weird
caching bandaid that only works in recovery?

I am not sure if it would be easy to detect all such failures and we
might end up opening other can of worms for us but if there is some
simpler way then sure we can consider it. OTOH, till we have a shared
cache of relation sizes (which I think is good for multiple things) it
seems the safe way to proceed by relying on the cache during recovery.
And, it is not that we can't change this once we have a shared
relation size solution.

The main argument I can think of against the idea of using plain old
smgrnblocks() is that the current error messages on smgrwrite()
failure for stray blocks would be indistinguishible from cases where
an external actor unlinked the file. I don't mind getting an error
that prevents checkpointing -- your system is in big trouble! -- but
it'd be nice to be able to detect that *we* unlinked the file,
implying the filesystem and bufferpool are out of sync, and spit out a
special diagnostic message. I suppose if it's the checkpointer doing
the writing, it could check if the relfilenode is on the
queued-up-for-delete-after-the-checkpoint list, and if so, it could
produce a different error message just for this edge case.
Unfortunately that's not a general solution, because any backend might
try to write a buffer out and they aren't synchronised with
checkpoints.

Yeah, but I am not sure if we can consider manual (external actor)
tinkering with the files the same as something that happened due to
the database server relying on the wrong information.

Here's a rough idea I thought of to detect this case; I'm not sure if
it has holes. When unlinking a relation, currently we truncate
segment 0 and unlink all the rest of the segments, and tell the
checkpointer to unlink segment 0 after the next checkpoint.

Do we always truncate all the blocks? What if the vacuum has cleaned
last N (say 100) blocks then how do we handle it?

--
With Regards,
Amit Kapila.

#161Thomas Munro
thomas.munro@gmail.com
In reply to: Amit Kapila (#160)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Nov 10, 2020 at 6:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

Do we always truncate all the blocks? What if the vacuum has cleaned
last N (say 100) blocks then how do we handle it?

Oh, hmm. Yeah, that idea only works for DROP, not for truncate last N.

#162tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#155)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

So I proceeded to update the patches using the "cached" parameter and
updated the corresponding comments to it in 0002.

OK, I'm in favor of the name "cached" now, although I first agreed with Horiguchi-san in that it's better to use a name that represents the nature (accurate) of information rather than the implementation (cached). Having a second thought, since smgr is a component that manages relation files on storage (file system), lseek(SEEK_END) is the accurate value for smgr. The cached value holds a possibly stale size up to which the relation has extended.

The patch looks almost good except for the minor ones:

(1)
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *accurate);

It's still accurate here.

(2)
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are already cached. See smgrnblocks() for more details.
+ *		This is only called in recovery when the block count of any fork is
+ *		cached and the total number of to-be-invalidated blocks per relation

count of any fork is
-> counts of all forks are

(3)
In 0004, I thought you would add the invalidated block counts of all relations to determine if the optimization is done, as Horiguchi-san suggested. But I find the current patch okay too.

Regards
Takayuki Tsunakawa

#163k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#162)
4 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Tuesday, November 10, 2020 3:10 PM, Tsunakawa-san wrote:

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

So I proceeded to update the patches using the "cached" parameter and
updated the corresponding comments to it in 0002.

OK, I'm in favor of the name "cached" now, although I first agreed with
Horiguchi-san in that it's better to use a name that represents the nature
(accurate) of information rather than the implementation (cached). Having
a second thought, since smgr is a component that manages relation files on
storage (file system), lseek(SEEK_END) is the accurate value for smgr. The
cached value holds a possibly stale size up to which the relation has
extended.

The patch looks almost good except for the minor ones:

Thank you for the review!

(1)
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber
forknum,
+							   bool *accurate);

It's still accurate here.

Already fixed in 0002.

(2)
+ *		This is only called in recovery when the block count of any
fork is
+ *		cached and the total number of to-be-invalidated blocks per
relation

count of any fork is
-> counts of all forks are

Fixed in 0003/

(3)
In 0004, I thought you would add the invalidated block counts of all relations
to determine if the optimization is done, as Horiguchi-san suggested. But I
find the current patch okay too.

Yeah, I found my approach easier to implement. The new change in 0004 is that
when entering the optimized path we now call FindAndDropRelFileNodeBuffers()
instead of DropRelFileNodeBuffers().

I have attached all the updated patches.
I'd appreciate your feedback.

Regards,
Kirk Jamison

Attachments:

v31-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v31-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From e43cb62883c7b7f770afa33ee5ac5e00738f9a9f Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v31 1/4] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v31-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v31-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From c74e0e236d9256c2b552b72063ce42d60b90a499 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v31 2/4] Add bool param in smgrnblocks() for cached blocks.

The flag ensures that we return a reliable value from smgrnblocks.
---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 21 ++++++++++++++++++---
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 34 insertions(+), 18 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 7e915bc..46fd97d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index d538f257..3874ff3 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -721,7 +721,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ad0d1a9..1680bf4 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -740,7 +740,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2857,7 +2857,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..e9dffd2 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -535,7 +535,6 @@ smgrwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 										buffer, skipFsync);
 }
 
-
 /*
  *	smgrwriteback() -- Trigger kernel writeback for the supplied range of
  *					   blocks.
@@ -551,18 +550,34 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 /*
  *	smgrnblocks() -- Calculate the number of blocks in the
  *					 supplied relation.
+ *
+ *		The "cached" flag ensures that no buffers exist for blocks after the
+ *		cached value is known to the process during recovery.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *cached)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  In recovery, the cached
+	 * value returned by the first lseek could be smaller than the actual number
+	 * of existing buffers of the file, which is caused by buggy Linux kernels
+	 * that might not have accounted for the recent write.  However, we can still
+	 * rely on the cached value even if we get a bogus value from first lseek
+	 * since it is impossible to have buffer for blocks after that file size.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (cached != NULL)
+			*cached = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
+
+	if (cached != NULL)
+		*cached = false;
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..cd99f1b 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *cached);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v31-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v31-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From a0169fd760a58543417aadbc880ae1ab6e7dc376 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Tue, 10 Nov 2020 06:27:11 +0000
Subject: [PATCH v31 3/4] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning. This improves the
performance especially when VACUUM or autovacuum truncated off any
of the empty pages at the end of relation.

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.  Otherwise, or if not in recovery, proceed to
sequential search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 115 +++++++++++++++++++++++++++++++++---
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 110 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 1680bf4..67284e3 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 256)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,11 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FindAndDropRelFileNodeBuffers(RelFileNode rnode,
+										  ForkNumber *forkNum,
+										  int nforks,
+										  BlockNumber *nForkBlocks,
+										  BlockNumber *firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2967,18 +2974,25 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		If the expected maximum number of buffers to be dropped is small
+ *		enough, individual buffer is located by BufTableLookup().  Otherwise,
+ *		the buffer pool is sequentially scanned. Since buffers must not be
+ *		left behind, the latter way is executed unless the sizes of all the
+ *		involved forks are already cached. See smgrnblocks() for more details.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		cached = false;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2992,6 +3006,35 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of to-be-invalidated blocks of a relation as
+	 * well as the total blocks for a given fork.  Give up the optimization
+	 * if the block is not cached.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &cached);
+
+		if (!cached)
+			break;
+
+		/* Get the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffers in the hashtable and drop them if the block size
+	 * is already cached and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		FindAndDropRelFileNodeBuffers(rnode.node, forkNum, nforks,
+									  nForkBlocks, firstDelBlock);
+		return;
+	}
+
 	for (i = 0; i < NBuffers; i++)
 	{
 		BufferDesc *bufHdr = GetBufferDescriptor(i);
@@ -3135,6 +3178,65 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 }
 
 /* ---------------------------------------------------------------------
+ *		FindAndDropRelFileNodeBuffers
+ *
+ *		This function finds and removes from the buffer pool all the pages
+ *		of the specified relation forks that have block numbers >= firstDelBlock.
+ *		(In particular, with firstDelBlock = 0, all pages are removed.)
+ *		This is only called in recovery when the block count of all forks are
+ *		cached and the total number of to-be-invalidated blocks per relation
+ *		do not exceed the threshold for full buffer scan.
+ * --------------------------------------------------------------------
+ */
+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+							  BlockNumber *nForkBlocks, BlockNumber *firstDelBlock)
+{
+	int		i;
+
+	for (i = 0; i < nforks; i++)
+	{
+		BlockNumber		curBlock;
+
+		for (curBlock = firstDelBlock[i]; curBlock < nForkBlocks[i]; curBlock++)
+		{
+			uint32		bufHash;		/* hash value for tag */
+			BufferTag	bufTag;			/* identity of requested block */
+			LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+			int		buf_id;
+			BufferDesc	*bufHdr;
+			uint32		buf_state;
+
+			/* create a tag so we can lookup the buffer */
+			INIT_BUFFERTAG(bufTag, rnode, forkNum[i], curBlock);
+
+			/* determine its hash code and partition lock ID */
+			bufHash = BufTableHashCode(&bufTag);
+			bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+			/* Check that it is in the buffer pool. If not, do nothing. */
+			LWLockAcquire(bufPartitionLock, LW_SHARED);
+			buf_id = BufTableLookup(&bufTag, bufHash);
+			LWLockRelease(bufPartitionLock);
+
+			if (buf_id < 0)
+				continue;
+
+			bufHdr = GetBufferDescriptor(buf_id);
+
+			buf_state = LockBufHdr(bufHdr);
+
+			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+				bufHdr->tag.forkNum == forkNum[i] &&
+				bufHdr->tag.blockNum >= firstDelBlock[i])
+				InvalidateBuffer(bufHdr);	/* releases spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);
+		}
+	}
+}
+
+/* ---------------------------------------------------------------------
  *		DropDatabaseBuffers
  *
  *		This function removes all the buffers in the buffer cache for a
@@ -3246,8 +3348,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index e9dffd2..9d3a67c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -605,7 +605,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v31-0004-TRUNCATE-optimization.patchapplication/octet-stream; name=v31-0004-TRUNCATE-optimization.patchDownload
From 882c3b8bf2641d7c02aafa8f94295fe9ec2c3a38 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Tue, 10 Nov 2020 08:24:04 +0000
Subject: [PATCH v31 4/4] TRUNCATE optimization.

DropRelFileNodesAllBuffers() is optimized to skip the time-consuming
scan of the whole buffer pool when the relation is small enough, or
when the number of blocks to be invalidated is below the full scan
threshold. This improves the performance when the TRUNCATE command
truncated off any of the empty pages at the end of relation.
---
 src/backend/storage/buffer/bufmgr.c | 91 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     | 14 +++---
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 90 insertions(+), 17 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 67284e3..f8be12c 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3086,17 +3086,34 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
+DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 {
 	int			i,
-				n = 0;
+				j,
+				nforks,
+				n = 0,
+				nrels = 0;
+	SMgrRelation	*rels;
+	RelFileNodeBackend *rnodes;
 	RelFileNode *nodes;
+	RelFileNode *undropped_nodes;
+	BlockNumber	nblocks[MAX_FORKNUM + 1];
+	BlockNumber	nBlocksToInvalidate;
+	BlockNumber	firstDelBlocks[MAX_FORKNUM + 1];
+	ForkNumber	forks[MAX_FORKNUM + 1];
 	bool		use_bsearch;
 
 	if (nnodes == 0)
 		return;
 
-	nodes = palloc(sizeof(RelFileNode) * nnodes);	/* non-local relations */
+	/* Create an array which contains all relations to be dropped */
+	rnodes = palloc(sizeof(RelFileNodeBackend) * nnodes);
+	for (i = 0; i < nnodes; i++)
+		rnodes[i] = smgr_reln[i]->smgr_rnode;
+
+	/* non-local relations */
+	rels = (SMgrRelation *)palloc(nnodes * sizeof(SMgrRelation));
+	nodes = palloc(sizeof(RelFileNode) * nnodes);
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	for (i = 0; i < nnodes; i++)
@@ -3107,7 +3124,10 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 				DropRelFileNodeAllLocalBuffers(rnodes[i].node);
 		}
 		else
+		{
+			rels[n] = smgr_reln[i];
 			nodes[n++] = rnodes[i].node;
+		}
 	}
 
 	/*
@@ -3117,6 +3137,57 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	if (n == 0)
 	{
 		pfree(nodes);
+		pfree(rels);
+		pfree(rnodes);
+		return;
+	}
+
+	/* undropped non-local relations */
+	undropped_nodes = palloc(sizeof(RelFileNode) * n);
+
+	/*
+	 * Zero the array of blocks because these will all be dropped anyway.
+	 * We don't care yet whether or not the block count of relation fork
+	 * is cached as it will be checked in DropRelFileNodeBuffers.  However,
+	 * we give up the optimization if the relation is large enough that the
+	 * number of blocks to be invalidated exceeds the threshold for full scan.
+	 */
+	memset(firstDelBlocks, 0, sizeof(firstDelBlocks));
+	for (i = 0; i < n; i++)
+	{
+		nforks = 0;
+		nBlocksToInvalidate = 0;
+
+		for (j = 0; j <= MAX_FORKNUM; j++)
+		{
+			if (!smgrexists(rels[i], j))
+				continue;
+
+			/* Get the number of blocks for a relation's fork */
+			nblocks[nforks] = smgrnblocks(rels[i], j, NULL);
+
+			nBlocksToInvalidate += nblocks[nforks];
+
+			forks[nforks++] = j;
+		}
+
+		if (nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+			FindAndDropRelFileNodeBuffers(nodes[i], forks, nforks, nblocks,
+										  firstDelBlocks);
+		else
+			undropped_nodes[nrels++] = nodes[i];
+	}
+
+	/*
+	 * If there are no undropped nodes, then we're done. Release the
+	 * memory and return.
+	 */
+	if (nrels == 0)
+	{
+		pfree(undropped_nodes);
+		pfree(nodes);
+		pfree(rels);
+		pfree(rnodes);
 		return;
 	}
 
@@ -3126,11 +3197,11 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	 * an exactly determined value, as it depends on many factors (CPU and RAM
 	 * speeds, amount of shared buffers etc.).
 	 */
-	use_bsearch = n > RELS_BSEARCH_THRESHOLD;
+	use_bsearch = nrels > RELS_BSEARCH_THRESHOLD;
 
 	/* sort the list of rnodes if necessary */
 	if (use_bsearch)
-		pg_qsort(nodes, n, sizeof(RelFileNode), rnode_comparator);
+		pg_qsort(undropped_nodes, nrels, sizeof(RelFileNode), rnode_comparator);
 
 	for (i = 0; i < NBuffers; i++)
 	{
@@ -3149,9 +3220,9 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 
 			for (j = 0; j < n; j++)
 			{
-				if (RelFileNodeEquals(bufHdr->tag.rnode, nodes[j]))
+				if (RelFileNodeEquals(bufHdr->tag.rnode, undropped_nodes[j]))
 				{
-					rnode = &nodes[j];
+					rnode = &undropped_nodes[j];
 					break;
 				}
 			}
@@ -3159,7 +3230,7 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 		else
 		{
 			rnode = bsearch((const void *) &(bufHdr->tag.rnode),
-							nodes, n, sizeof(RelFileNode),
+							undropped_nodes, nrels, sizeof(RelFileNode),
 							rnode_comparator);
 		}
 
@@ -3173,8 +3244,10 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 		else
 			UnlockBufHdr(bufHdr, buf_state);
 	}
-
+	pfree(undropped_nodes);
 	pfree(nodes);
+	pfree(rels);
+	pfree(rnodes);
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 9d3a67c..3663bb7 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -392,7 +392,13 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 		return;
 
 	/*
-	 * create an array which contains all relations to be dropped, and close
+	 * Get rid of any remaining buffers for the relations.  bufmgr will just
+	 * drop them without bothering to write the contents.
+	 */
+	DropRelFileNodesAllBuffers(rels, nrels);
+
+	/*
+	 * Create an array which contains all relations to be dropped, and close
 	 * each relation's forks at the smgr level while at it
 	 */
 	rnodes = palloc(sizeof(RelFileNodeBackend) * nrels);
@@ -409,12 +415,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	}
 
 	/*
-	 * Get rid of any remaining buffers for the relations.  bufmgr will just
-	 * drop them without bothering to write the contents.
-	 */
-	DropRelFileNodesAllBuffers(rnodes, nrels);
-
-	/*
 	 * It'd be nice to tell the stats collector to forget them immediately,
 	 * too. But we can't because we don't know the OIDs.
 	 */
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 056f65e..2e5189b 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -205,7 +205,7 @@ extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels)
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
-extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
+extern void DropRelFileNodesAllBuffers(struct SMgrRelationData **smgr_reln, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
 #define RelationGetNumberOfBlocks(reln) \
-- 
1.8.3.1

#164tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#163)
RE: [Patch] Optimize dropping of relation buffers using dlist

The patch looks OK. I think as Thomas-san suggested, we can remove the modification to smgrnblocks() and don't care wheter the size is cached or not. But I think the current patch is good too, so I'd like to leave it up to a committer to decide which to choose.

I measured performance in a different angle -- the time DropRelFileNodeBuffers() and DropRelFileNodeAllBuffers() took. That reveals the direct improvement and degradation.

I used 1,000 tables, each of which is 1 MB. I used shared_buffers = 128 MB for the case where the traditional full buffer scan is done, and shared_buffers = 100 GB for the case where the optimization path takes effect.

The results are almost good as follows:

A. UNPATCHED

128 MB shared_buffers
1. VACUUM = 0.04 seconds
2. TRUNCATE = 0.04 seconds

100 GB shared_buffers
3. VACUUM = 69.4 seconds
4. TRUNCATE = 69.1 seconds

B. PATCHED

128 MB shared_buffers (full scan)
5. VACUUM = 0.04 seconds
6. TRUNCATE = 0.07 seconds

100 GB shared_buffers (optimized path)
7. VACUUM = 0.02 seconds
8. TRUNCATE = 0.08 seconds

So, I'd like to mark this as ready for committer.

Regards
Takayuki Tsunakawa

#165k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#164)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, November 12, 2020 1:14 PM, Tsunakawa-san wrote:

The patch looks OK. I think as Thomas-san suggested, we can remove the
modification to smgrnblocks() and don't care wheter the size is cached or not.
But I think the current patch is good too, so I'd like to leave it up to a
committer to decide which to choose.
I measured performance in a different angle -- the time
DropRelFileNodeBuffers() and DropRelFileNodeAllBuffers() took. That
reveals the direct improvement and degradation.

I used 1,000 tables, each of which is 1 MB. I used shared_buffers = 128 MB
for the case where the traditional full buffer scan is done, and shared_buffers
= 100 GB for the case where the optimization path takes effect.

The results are almost good as follows:

A. UNPATCHED

128 MB shared_buffers
1. VACUUM = 0.04 seconds
2. TRUNCATE = 0.04 seconds

100 GB shared_buffers
3. VACUUM = 69.4 seconds
4. TRUNCATE = 69.1 seconds

B. PATCHED

128 MB shared_buffers (full scan)
5. VACUUM = 0.04 seconds
6. TRUNCATE = 0.07 seconds

100 GB shared_buffers (optimized path)
7. VACUUM = 0.02 seconds
8. TRUNCATE = 0.08 seconds

So, I'd like to mark this as ready for committer.

I forgot to reply.
Thank you very much Tsunakawa-san for testing and to everyone
who has provided their reviews and insights as well.

Now thinking about smgrnblocks(), currently Thomas Munro is also working on implementing a
shared SmgrRelation [1]/messages/by-id/CA+hUKG+7Ok26MHiFWVEiAy2UMgHkrCieycQ1eFdA=t2JTfUgwA@mail.gmail.com to store sizes. However, since that is still under development and the
discussion is still ongoing, I hope we can first commit these set of patches here as these are already
in committable form. I think it's alright to accept the early improvements implemented in this thread
to the source code.

[1]: /messages/by-id/CA+hUKG+7Ok26MHiFWVEiAy2UMgHkrCieycQ1eFdA=t2JTfUgwA@mail.gmail.com

Regards,
Kirk Jamison

#166Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#165)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Nov 18, 2020 at 2:34 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Thursday, November 12, 2020 1:14 PM, Tsunakawa-san wrote:
I forgot to reply.
Thank you very much Tsunakawa-san for testing and to everyone
who has provided their reviews and insights as well.

Now thinking about smgrnblocks(), currently Thomas Munro is also working on implementing a
shared SmgrRelation [1] to store sizes. However, since that is still under development and the
discussion is still ongoing, I hope we can first commit these set of patches here as these are already
in committable form. I think it's alright to accept the early improvements implemented in this thread
to the source code.

Yeah, that won't be a bad idea especially because the patch being
discussed in the thread you referred is still in an exploratory phase.
I haven't tested or done a detailed review but I feel there shouldn't
be many problems if we agree on the approach.

Thomas/others, do you have objections to proceeding here? It shouldn't
be a big problem to change the code in this area even if we get the
shared relation size stuff in.

--
With Regards,
Amit Kapila.

#167Andres Freund
andres@anarazel.de
In reply to: Amit Kapila (#166)
Re: [Patch] Optimize dropping of relation buffers using dlist

Hi,

On 2020-11-18 17:34:31 +0530, Amit Kapila wrote:

Yeah, that won't be a bad idea especially because the patch being
discussed in the thread you referred is still in an exploratory phase.
I haven't tested or done a detailed review but I feel there shouldn't
be many problems if we agree on the approach.

Thomas/others, do you have objections to proceeding here? It shouldn't
be a big problem to change the code in this area even if we get the
shared relation size stuff in.

I'm doubtful the patches as is are a good idea / address the correctness
concerns to a sufficient degree.

One important part of that is that the patch includes pretty much zero
explanations about why it is safe what it is doing. Something having
being discussed deep in this thread won't help us in a few months, not
to speak of a few years.

The commit message says:

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.

and the code only applies the optimized scan only when cached:
+	/*
+	 * Look up the buffers in the hashtable and drop them if the block size
+	 * is already cached and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)

This seems quite narrow to me. There's plenty cases where there's no
cached relation size in the startup process, restricting the
availability of this optimization as written. Where do we even use
DropRelFileNodeBuffers() in recovery? The most common path is
DropRelationFiles()->smgrdounlinkall()->DropRelFileNodesAllBuffers(),
which 3/4 doesn't address and 4/4 doesn't mention.

4/4 seems to address DropRelationFiles(), but only talks about TRUNCATE?

I'm also worried about the cases where this could cause buffers left in
the buffer pool, without a crosscheck like Thomas' patch would allow to
add. Obviously other processes can dirty buffers in hot_standby, so any
leftover buffer could have bad consequences.

I also don't get why 4/4 would be a good idea on its own. It uses
BUF_DROP_FULL_SCAN_THRESHOLD to guard FindAndDropRelFileNodeBuffers() on
a per relation basis. But since DropRelFileNodesAllBuffers() can be used
for many relations at once, this could end up doing
BUF_DROP_FULL_SCAN_THRESHOLD - 1 lookups a lot of times, once for each
of nnodes relations?

Also, how is 4/4 safe - this is outside of recovery too?

Smaller comment:

+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+							  BlockNumber *nForkBlocks, BlockNumber *firstDelBlock)
...
+			/* Check that it is in the buffer pool. If not, do nothing. */
+			LWLockAcquire(bufPartitionLock, LW_SHARED);
+			buf_id = BufTableLookup(&bufTag, bufHash);
...
+			bufHdr = GetBufferDescriptor(buf_id);
+
+			buf_state = LockBufHdr(bufHdr);
+
+			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+				bufHdr->tag.forkNum == forkNum[i] &&
+				bufHdr->tag.blockNum >= firstDelBlock[i])
+				InvalidateBuffer(bufHdr);	/* releases spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);
a

I'm a bit confused about the check here. We hold a buffer partition
lock, and have done a lookup in the mapping table. Why are we then
rechecking the relfilenode/fork/blocknum? And why are we doing so
holding the buffer header lock, which is essentially a spinlock, so
should only ever be held for very short portions?

This looks like it's copying logic from DropRelFileNodeBuffers() etc,
but there the situation is different: We haven't done a buffer mapping
lookup, and we don't hold a partition lock!

Greetings,

Andres Freund

#168Amit Kapila
amit.kapila16@gmail.com
In reply to: Andres Freund (#167)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Nov 18, 2020 at 11:43 PM Andres Freund <andres@anarazel.de> wrote:

Hi,

On 2020-11-18 17:34:31 +0530, Amit Kapila wrote:

Yeah, that won't be a bad idea especially because the patch being
discussed in the thread you referred is still in an exploratory phase.
I haven't tested or done a detailed review but I feel there shouldn't
be many problems if we agree on the approach.

Thomas/others, do you have objections to proceeding here? It shouldn't
be a big problem to change the code in this area even if we get the
shared relation size stuff in.

I'm doubtful the patches as is are a good idea / address the correctness
concerns to a sufficient degree.

One important part of that is that the patch includes pretty much zero
explanations about why it is safe what it is doing. Something having
being discussed deep in this thread won't help us in a few months, not
to speak of a few years.

The commit message says:

While recovery, we can get a reliable cached value of nblocks for
supplied relation's fork, and it's safe because there are no other
processes but the startup process that changes the relation size
during recovery.

and the code only applies the optimized scan only when cached:
+       /*
+        * Look up the buffers in the hashtable and drop them if the block size
+        * is already cached and the total blocks to be invalidated is below the
+        * full scan threshold.  Otherwise, give up the optimization.
+        */
+       if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)

This seems quite narrow to me. There's plenty cases where there's no
cached relation size in the startup process, restricting the
availability of this optimization as written. Where do we even use
DropRelFileNodeBuffers() in recovery?

This will be used in the recovery of truncate done by vacuum (via
replay of XLOG_SMGR_TRUNCATE->smgrtruncate->DropRelFileNodeBuffers).
And Kirk-San has done some testing [1]/messages/by-id/OSBPR01MB23413F14ED6B2D0D007698F4EFED0@OSBPR01MB2341.jpnprd01.prod.outlook.com[2]/messages/by-id/OSBPR01MB234176B1829AECFE9FDDFCC2EFE90@OSBPR01MB2341.jpnprd01.prod.outlook.com to show the performance
benefits of the same.

The most common path is
DropRelationFiles()->smgrdounlinkall()->DropRelFileNodesAllBuffers(),
which 3/4 doesn't address and 4/4 doesn't mention.

4/4 seems to address DropRelationFiles(), but only talks about TRUNCATE?

I'm also worried about the cases where this could cause buffers left in
the buffer pool, without a crosscheck like Thomas' patch would allow to
add. Obviously other processes can dirty buffers in hot_standby, so any
leftover buffer could have bad consequences.

The problem can only arise if other processes extend the relation. The
idea was that in recovery it extends relation by one process which
helps to maintain the cache. Kirk seems to have done testing to
cross-verify it by using his first patch
(Prevent-invalidating-blocks-in-smgrextend-during). Which other
crosscheck you are referring here?

I agree that we can do a better job by expanding comments to clearly
state why it is safe.

[1]: /messages/by-id/OSBPR01MB23413F14ED6B2D0D007698F4EFED0@OSBPR01MB2341.jpnprd01.prod.outlook.com
[2]: /messages/by-id/OSBPR01MB234176B1829AECFE9FDDFCC2EFE90@OSBPR01MB2341.jpnprd01.prod.outlook.com

--
With Regards,
Amit Kapila.

#169tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Andres Freund (#167)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Andres Freund <andres@anarazel.de>

DropRelFileNodeBuffers() in recovery? The most common path is
DropRelationFiles()->smgrdounlinkall()->DropRelFileNodesAllBuffers(),
which 3/4 doesn't address and 4/4 doesn't mention.

4/4 seems to address DropRelationFiles(), but only talks about TRUNCATE?

Yes. DropRelationFiles() is used in the following two paths:

[Replay of TRUNCATE during recovery]
xact_redo_commit/abort() -> DropRelationFiles()
-> smgrdounlinkall() -> DropRelFileNodesAllBuffers()

[COMMIT/ROLLBACK PREPARED]
FinishPreparedTransaction() -> DropRelationFiles()
-> smgrdounlinkall() -> DropRelFileNodesAllBuffers()

I also don't get why 4/4 would be a good idea on its own. It uses
BUF_DROP_FULL_SCAN_THRESHOLD to guard
FindAndDropRelFileNodeBuffers() on a per relation basis. But since
DropRelFileNodesAllBuffers() can be used for many relations at once, this
could end up doing BUF_DROP_FULL_SCAN_THRESHOLD - 1 lookups a lot of
times, once for each of nnodes relations?

So, the threshold value should be compared with the total number of blocks of all target relations, not each relation. You seem to be right, got it.

Also, how is 4/4 safe - this is outside of recovery too?

It seems that DropRelFileNodesAllBuffers() should trigger the new optimization path only when InRecovery == true, because it intentionally doesn't check the "accurate" value returned from smgrnblocks().

Smaller comment:

+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber *forkNum,
int nforks,
+							  BlockNumber
*nForkBlocks, BlockNumber *firstDelBlock)
...
+			/* Check that it is in the buffer pool. If not, do nothing.
*/
+			LWLockAcquire(bufPartitionLock, LW_SHARED);
+			buf_id = BufTableLookup(&bufTag, bufHash);
...
+			bufHdr = GetBufferDescriptor(buf_id);
+
+			buf_state = LockBufHdr(bufHdr);
+
+			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+				bufHdr->tag.forkNum == forkNum[i] &&
+				bufHdr->tag.blockNum >= firstDelBlock[i])
+				InvalidateBuffer(bufHdr);	/* releases
spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);

I'm a bit confused about the check here. We hold a buffer partition lock, and
have done a lookup in the mapping table. Why are we then rechecking the
relfilenode/fork/blocknum? And why are we doing so holding the buffer header
lock, which is essentially a spinlock, so should only ever be held for very short
portions?

This looks like it's copying logic from DropRelFileNodeBuffers() etc, but there
the situation is different: We haven't done a buffer mapping lookup, and we
don't hold a partition lock!

That's because the buffer partition lock is released immediately after the hash table has been looked up. As an aside, InvalidateBuffer() requires the caller to hold the buffer header spinlock and doesn't hold the buffer partition lock.

Regards
Takayuki Tsunakawa

#170k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#169)
4 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, November 19, 2020 4:08 PM, Tsunakawa, Takayuki wrote:

From: Andres Freund <andres@anarazel.de>

DropRelFileNodeBuffers() in recovery? The most common path is
DropRelationFiles()->smgrdounlinkall()->DropRelFileNodesAllBuffers(),
which 3/4 doesn't address and 4/4 doesn't mention.

4/4 seems to address DropRelationFiles(), but only talks about

TRUNCATE?

Yes. DropRelationFiles() is used in the following two paths:

[Replay of TRUNCATE during recovery]
xact_redo_commit/abort() -> DropRelationFiles() -> smgrdounlinkall() ->
DropRelFileNodesAllBuffers()

[COMMIT/ROLLBACK PREPARED]
FinishPreparedTransaction() -> DropRelationFiles() -> smgrdounlinkall()
-> DropRelFileNodesAllBuffers()

Yes. The concern is that it was not clear in the function descriptions and commit logs
what the optimizations for the DropRelFileNodeBuffers() and DropRelFileNodesAllBuffers()
are for. So I revised only the function description of DropRelFileNodeBuffers() and the
commit logs of the 0003-0004 patches. Please check if the brief descriptions would suffice.

I also don't get why 4/4 would be a good idea on its own. It uses
BUF_DROP_FULL_SCAN_THRESHOLD to guard
FindAndDropRelFileNodeBuffers() on a per relation basis. But since
DropRelFileNodesAllBuffers() can be used for many relations at once,
this could end up doing BUF_DROP_FULL_SCAN_THRESHOLD - 1

lookups a lot

of times, once for each of nnodes relations?

So, the threshold value should be compared with the total number of blocks
of all target relations, not each relation. You seem to be right, got it.

Fixed this in 0004 patch. Now we compare the total number of buffers-to-be-invalidated
For ALL relations to the BUF_DROP_FULL_SCAN_THRESHOLD.

Also, how is 4/4 safe - this is outside of recovery too?

It seems that DropRelFileNodesAllBuffers() should trigger the new
optimization path only when InRecovery == true, because it intentionally
doesn't check the "accurate" value returned from smgrnblocks().

Fixed it in 0004 patch. Now we ensure that we only enter the optimization path
Iff during recovery.

From: Amit Kapila <amit.kapila16@gmail.com>
On Wed, Nov 18, 2020 at 11:43 PM Andres Freund <andres@anarazel.de>

I'm also worried about the cases where this could cause buffers left
in the buffer pool, without a crosscheck like Thomas' patch would
allow to add. Obviously other processes can dirty buffers in
hot_standby, so any leftover buffer could have bad consequences.

The problem can only arise if other processes extend the relation. The idea
was that in recovery it extends relation by one process which helps to
maintain the cache. Kirk seems to have done testing to cross-verify it by using
his first patch (Prevent-invalidating-blocks-in-smgrextend-during). Which
other crosscheck you are referring here?

I agree that we can do a better job by expanding comments to clearly state
why it is safe.

Yes, basically what Amit-san also mentioned above. The first patch prevents that.
And in the description of DropRelFileNodeBuffers in the 0003 patch, please check
If that would suffice.

Smaller comment:

+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber

*forkNum,

int nforks,
+							  BlockNumber
*nForkBlocks, BlockNumber *firstDelBlock) ...
+			/* Check that it is in the buffer pool. If not, do

nothing.

*/
+			LWLockAcquire(bufPartitionLock, LW_SHARED);
+			buf_id = BufTableLookup(&bufTag, bufHash);
...
+			bufHdr = GetBufferDescriptor(buf_id);
+
+			buf_state = LockBufHdr(bufHdr);
+
+			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode)

&&

+				bufHdr->tag.forkNum == forkNum[i] &&
+				bufHdr->tag.blockNum >= firstDelBlock[i])
+				InvalidateBuffer(bufHdr);	/* releases
spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);

I'm a bit confused about the check here. We hold a buffer partition
lock, and have done a lookup in the mapping table. Why are we then
rechecking the relfilenode/fork/blocknum? And why are we doing so
holding the buffer header lock, which is essentially a spinlock, so
should only ever be held for very short portions?

This looks like it's copying logic from DropRelFileNodeBuffers() etc,
but there the situation is different: We haven't done a buffer mapping
lookup, and we don't hold a partition lock!

That's because the buffer partition lock is released immediately after the hash
table has been looked up. As an aside, InvalidateBuffer() requires the caller
to hold the buffer header spinlock and doesn't hold the buffer partition lock.

Yes. Holding the buffer header spinlock is necessary to invalidate the buffers.
As for buffer mapping partition lock, as mentioned by Tsunakawa-san, it is
released immediately after BufTableLookup, which is similar to lookup done in
PrefetchSharedBuffer. So I retained these changes.

I have attached the updated patches. Aside from descriptions, no other major
changes in the patch set except 0004. Feedbacks are welcome.

Regards,
Kirk Jamison

Attachments:

v32-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v32-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From 813aac247d8779074da764e741934c3115288753 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v32 1/4] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v32-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v32-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From b14fb69b96f5a9f18ed9dc322a6cd4254c411945 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v32 2/4] Add bool param in smgrnblocks() for cached blocks.

The flag ensures that we return a reliable value from smgrnblocks.
---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 21 ++++++++++++++++++---
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 34 insertions(+), 18 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 32a3099..afc640d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index d538f257..3874ff3 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -721,7 +721,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ad0d1a9..1680bf4 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -740,7 +740,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2857,7 +2857,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..e9dffd2 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -535,7 +535,6 @@ smgrwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 										buffer, skipFsync);
 }
 
-
 /*
  *	smgrwriteback() -- Trigger kernel writeback for the supplied range of
  *					   blocks.
@@ -551,18 +550,34 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 /*
  *	smgrnblocks() -- Calculate the number of blocks in the
  *					 supplied relation.
+ *
+ *		The "cached" flag ensures that no buffers exist for blocks after the
+ *		cached value is known to the process during recovery.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *cached)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  In recovery, the cached
+	 * value returned by the first lseek could be smaller than the actual number
+	 * of existing buffers of the file, which is caused by buggy Linux kernels
+	 * that might not have accounted for the recent write.  However, we can still
+	 * rely on the cached value even if we get a bogus value from first lseek
+	 * since it is impossible to have buffer for blocks after that file size.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (cached != NULL)
+			*cached = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
+
+	if (cached != NULL)
+		*cached = false;
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..cd99f1b 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *cached);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v32-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v32-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 009c65320ce2636a39ba8649bc2d600635c36851 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Tue, 10 Nov 2020 06:27:11 +0000
Subject: [PATCH v32 3/4] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning. This improves the
performance especially when VACUUM or autovacuum truncated off any
of the empty pages at the end of relation.

While recovery, when WAL files of XLOG_SMGR_TRUNCATE from vacuum
or autovacuum are replayed, the buffers are dropped when the sizes
of all involved forks of a relation are already "cached". We can get
a reliable size of nblocks for supplied relation's fork at that time,
and it's safe because DropRelFileNodeBuffers() relies on the behavior
that cached nblocks will not be invalidated by file extension during
recovery.  Otherwise, or if not in recovery, proceed to sequential
search of the whole buffer pool.
---
 src/backend/storage/buffer/bufmgr.c | 118 +++++++++++++++++++++++++++++++++---
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 113 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 1680bf4..f354eb4 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 256)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,11 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FindAndDropRelFileNodeBuffers(RelFileNode rnode,
+										  ForkNumber *forkNum,
+										  int nforks,
+										  BlockNumber *nForkBlocks,
+										  BlockNumber *firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2967,18 +2974,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		While in recovery, if the expected maximum number of buffers to be
+ *		dropped is small enough and the sizes of all involved forks are
+ *		already cached, individual buffer is located by BufTableLookup().
+ *		It is safe because cached blocks will not be invalidated by file
+ *		extension during recovery.  See smgrnblocks() and smgrextend() for
+ *		more details. Otherwise, if the conditions for optimization are not
+ *		met, the buffer pool is sequentially scanned so that no buffers are
+ *		left behind.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		cached = false;
+	BlockNumber	nForkBlocks[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2992,6 +3009,35 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of to-be-invalidated blocks of a relation as
+	 * well as the total blocks for a given fork.  Give up the optimization
+	 * if the block is not cached.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlocks[i] = smgrnblocks(smgr_reln, forkNum[i], &cached);
+
+		if (!cached)
+			break;
+
+		/* Get the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlocks[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffers in the hashtable and drop them if the block size
+	 * is already cached and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		FindAndDropRelFileNodeBuffers(rnode.node, forkNum, nforks,
+									  nForkBlocks, firstDelBlock);
+		return;
+	}
+
 	for (i = 0; i < NBuffers; i++)
 	{
 		BufferDesc *bufHdr = GetBufferDescriptor(i);
@@ -3135,6 +3181,65 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 }
 
 /* ---------------------------------------------------------------------
+ *		FindAndDropRelFileNodeBuffers
+ *
+ *		This function finds and removes from the buffer pool all the pages
+ *		of the specified relation forks that have block numbers >= firstDelBlock.
+ *		(In particular, with firstDelBlock = 0, all pages are removed.)
+ *		This is only called in recovery when the block count of all forks are
+ *		cached and the total number of to-be-invalidated blocks per relation
+ *		does not exceed the threshold for full buffer scan.
+ * --------------------------------------------------------------------
+ */
+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber *forkNum, int nforks,
+							  BlockNumber *nForkBlocks, BlockNumber *firstDelBlock)
+{
+	int		i;
+
+	for (i = 0; i < nforks; i++)
+	{
+		BlockNumber		curBlock;
+
+		for (curBlock = firstDelBlock[i]; curBlock < nForkBlocks[i]; curBlock++)
+		{
+			uint32		bufHash;		/* hash value for tag */
+			BufferTag	bufTag;			/* identity of requested block */
+			LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+			int		buf_id;
+			BufferDesc	*bufHdr;
+			uint32		buf_state;
+
+			/* create a tag so we can lookup the buffer */
+			INIT_BUFFERTAG(bufTag, rnode, forkNum[i], curBlock);
+
+			/* determine its hash code and partition lock ID */
+			bufHash = BufTableHashCode(&bufTag);
+			bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+			/* Check that it is in the buffer pool. If not, do nothing. */
+			LWLockAcquire(bufPartitionLock, LW_SHARED);
+			buf_id = BufTableLookup(&bufTag, bufHash);
+			LWLockRelease(bufPartitionLock);
+
+			if (buf_id < 0)
+				continue;
+
+			bufHdr = GetBufferDescriptor(buf_id);
+
+			buf_state = LockBufHdr(bufHdr);
+
+			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+				bufHdr->tag.forkNum == forkNum[i] &&
+				bufHdr->tag.blockNum >= firstDelBlock[i])
+				InvalidateBuffer(bufHdr);	/* releases spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);
+		}
+	}
+}
+
+/* ---------------------------------------------------------------------
  *		DropDatabaseBuffers
  *
  *		This function removes all the buffers in the buffer cache for a
@@ -3246,8 +3351,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index e9dffd2..9d3a67c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -605,7 +605,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v32-0004-Optimize-DropRelFileNodesAllBuffers-in-recovery.patchapplication/octet-stream; name=v32-0004-Optimize-DropRelFileNodesAllBuffers-in-recovery.patchDownload
From 5beaa843a9c96e36b75c32cd6efba49fbe5b48cd Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Thu, 26 Nov 2020 02:42:58 +0000
Subject: [PATCH v32 4/4] Optimize DropRelFileNodesAllBuffers() in recovery.

DropRelFileNodesAllBuffers() is optimized to skip the time-consuming
scan of the whole buffer pool during recovery when the relation is
small enough, or when the number of blocks to be invalidated is below
the full scan threshold. This improves the DropRelationFiles()
performance when the TRUNCATE command truncated off any of the empty
pages at the end of relation, and when dropping relation buffers if a
commit/rollback transaction has been prepared in FinishPreparedTransaction().
---
 src/backend/storage/buffer/bufmgr.c | 95 ++++++++++++++++++++++++++++++++++++-
 src/backend/storage/smgr/smgr.c     | 14 +++---
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 101 insertions(+), 10 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index f354eb4..143f592 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3089,17 +3089,32 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
+DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 {
 	int			i,
+				j,
+				*nforks,
 				n = 0;
+	SMgrRelation	*rels;
+	RelFileNodeBackend *rnodes;
 	RelFileNode *nodes;
+	BlockNumber	**blocks;
+	BlockNumber	*firstDelBlocks;
+	BlockNumber	nBlocksToInvalidate = 0;
+	ForkNumber	**forks;
 	bool		use_bsearch;
 
 	if (nnodes == 0)
 		return;
 
-	nodes = palloc(sizeof(RelFileNode) * nnodes);	/* non-local relations */
+	/* Create an array which contains all relations to be dropped */
+	rnodes = palloc(sizeof(RelFileNodeBackend) * nnodes);
+	for (i = 0; i < nnodes; i++)
+		rnodes[i] = smgr_reln[i]->smgr_rnode;
+
+	/* non-local relations */
+	rels = palloc(sizeof(SMgrRelation) * nnodes);
+	nodes = palloc(sizeof(RelFileNode) * nnodes);
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	for (i = 0; i < nnodes; i++)
@@ -3110,7 +3125,10 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 				DropRelFileNodeAllLocalBuffers(rnodes[i].node);
 		}
 		else
+		{
+			rels[n] = smgr_reln[i];
 			nodes[n++] = rnodes[i].node;
+		}
 	}
 
 	/*
@@ -3120,6 +3138,68 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	if (n == 0)
 	{
 		pfree(nodes);
+		pfree(rels);
+		pfree(rnodes);
+		return;
+	}
+
+	nforks = palloc(sizeof(int) * n);
+	forks = palloc(sizeof(ForkNumber *) * n);
+	blocks = palloc(sizeof(BlockNumber *) * n);
+	firstDelBlocks = palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));
+	for (i = 0; i < n; i++)
+	{
+		forks[i] = palloc(sizeof(ForkNumber) * (MAX_FORKNUM + 1));
+		blocks[i] = palloc(sizeof(BlockNumber) * (MAX_FORKNUM + 1));
+	}
+
+	for (i = 0; i < n; i++)
+	{
+		int		numForks = 0;
+
+		for (j = 0; j <= MAX_FORKNUM; j++)
+		{
+			if (!smgrexists(rels[i], j))
+				continue;
+
+			/* Get the number of blocks for a relation's fork */
+			blocks[i][numForks] = smgrnblocks(rels[i], j, NULL);
+
+			nBlocksToInvalidate += blocks[i][numForks];
+
+			forks[i][numForks++] = j;
+		}
+		nforks[i] = numForks;
+	}
+
+	/* Zero the array of blocks because these will all be dropped anyway */
+	MemSet(firstDelBlocks, 0, sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));
+
+	/*
+	 * We enter the optimization iff we are in recovery and the number of blocks to
+	 * be invalidated for all relations do not exceed the threshold for full scan.
+	 * Otherwise, we proceed to full scan of the whole buffer pool.
+	 */
+	if (InRecovery && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < n; j++)
+		{
+			FindAndDropRelFileNodeBuffers(nodes[j], forks[j], nforks[j],
+										  blocks[j], firstDelBlocks);
+		}
+
+		for (j = 0; j < n; j++)
+		{
+			pfree(blocks[j]);
+			pfree(forks[j]);
+		}
+		pfree(firstDelBlocks);
+		pfree(blocks);
+		pfree(forks);
+		pfree(nforks);
+		pfree(nodes);
+		pfree(rels);
+		pfree(rnodes);
 		return;
 	}
 
@@ -3177,7 +3257,18 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 			UnlockBufHdr(bufHdr, buf_state);
 	}
 
+	for (i = 0; i < n; i++)
+	{
+		pfree(blocks[i]);
+		pfree(forks[i]);
+	}
+	pfree(firstDelBlocks);
+	pfree(blocks);
+	pfree(forks);
+	pfree(nforks);
 	pfree(nodes);
+	pfree(rels);
+	pfree(rnodes);
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 9d3a67c..3663bb7 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -392,7 +392,13 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 		return;
 
 	/*
-	 * create an array which contains all relations to be dropped, and close
+	 * Get rid of any remaining buffers for the relations.  bufmgr will just
+	 * drop them without bothering to write the contents.
+	 */
+	DropRelFileNodesAllBuffers(rels, nrels);
+
+	/*
+	 * Create an array which contains all relations to be dropped, and close
 	 * each relation's forks at the smgr level while at it
 	 */
 	rnodes = palloc(sizeof(RelFileNodeBackend) * nrels);
@@ -409,12 +415,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	}
 
 	/*
-	 * Get rid of any remaining buffers for the relations.  bufmgr will just
-	 * drop them without bothering to write the contents.
-	 */
-	DropRelFileNodesAllBuffers(rnodes, nrels);
-
-	/*
 	 * It'd be nice to tell the stats collector to forget them immediately,
 	 * too. But we can't because we don't know the OIDs.
 	 */
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 056f65e..2e5189b 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -205,7 +205,7 @@ extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels)
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
-extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
+extern void DropRelFileNodesAllBuffers(struct SMgrRelationData **smgr_reln, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
 #define RelationGetNumberOfBlocks(reln) \
-- 
1.8.3.1

#171k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#170)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: k.jamison@fujitsu.com <k.jamison@fujitsu.com>
On Thursday, November 19, 2020 4:08 PM, Tsunakawa, Takayuki wrote:

From: Andres Freund <andres@anarazel.de>

DropRelFileNodeBuffers() in recovery? The most common path is
DropRelationFiles()->smgrdounlinkall()->DropRelFileNodesAllBuffers()
, which 3/4 doesn't address and 4/4 doesn't mention.

4/4 seems to address DropRelationFiles(), but only talks about

TRUNCATE?

Yes. DropRelationFiles() is used in the following two paths:

[Replay of TRUNCATE during recovery]
xact_redo_commit/abort() -> DropRelationFiles() -> smgrdounlinkall()
->
DropRelFileNodesAllBuffers()

[COMMIT/ROLLBACK PREPARED]
FinishPreparedTransaction() -> DropRelationFiles() ->
smgrdounlinkall()
-> DropRelFileNodesAllBuffers()

Yes. The concern is that it was not clear in the function descriptions and
commit logs what the optimizations for the DropRelFileNodeBuffers() and
DropRelFileNodesAllBuffers() are for. So I revised only the function
description of DropRelFileNodeBuffers() and the commit logs of the
0003-0004 patches. Please check if the brief descriptions would suffice.

I also don't get why 4/4 would be a good idea on its own. It uses
BUF_DROP_FULL_SCAN_THRESHOLD to guard
FindAndDropRelFileNodeBuffers() on a per relation basis. But since
DropRelFileNodesAllBuffers() can be used for many relations at once,
this could end up doing BUF_DROP_FULL_SCAN_THRESHOLD - 1

lookups a lot

of times, once for each of nnodes relations?

So, the threshold value should be compared with the total number of
blocks of all target relations, not each relation. You seem to be right, got it.

Fixed this in 0004 patch. Now we compare the total number of
buffers-to-be-invalidated For ALL relations to the
BUF_DROP_FULL_SCAN_THRESHOLD.

Also, how is 4/4 safe - this is outside of recovery too?

It seems that DropRelFileNodesAllBuffers() should trigger the new
optimization path only when InRecovery == true, because it
intentionally doesn't check the "accurate" value returned from

smgrnblocks().

Fixed it in 0004 patch. Now we ensure that we only enter the optimization path
Iff during recovery.

From: Amit Kapila <amit.kapila16@gmail.com> On Wed, Nov 18, 2020 at
11:43 PM Andres Freund <andres@anarazel.de>

I'm also worried about the cases where this could cause buffers left
in the buffer pool, without a crosscheck like Thomas' patch would
allow to add. Obviously other processes can dirty buffers in
hot_standby, so any leftover buffer could have bad consequences.

The problem can only arise if other processes extend the relation. The
idea was that in recovery it extends relation by one process which
helps to maintain the cache. Kirk seems to have done testing to
cross-verify it by using his first patch
(Prevent-invalidating-blocks-in-smgrextend-during). Which other

crosscheck you are referring here?

I agree that we can do a better job by expanding comments to clearly
state why it is safe.

Yes, basically what Amit-san also mentioned above. The first patch prevents
that.
And in the description of DropRelFileNodeBuffers in the 0003 patch, please
check If that would suffice.

Smaller comment:

+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber

*forkNum,

int nforks,
+							  BlockNumber
*nForkBlocks, BlockNumber *firstDelBlock) ...
+			/* Check that it is in the buffer pool. If not, do

nothing.

*/
+			LWLockAcquire(bufPartitionLock, LW_SHARED);
+			buf_id = BufTableLookup(&bufTag, bufHash);
...
+			bufHdr = GetBufferDescriptor(buf_id);
+
+			buf_state = LockBufHdr(bufHdr);
+
+			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode)

&&

+				bufHdr->tag.forkNum == forkNum[i] &&
+				bufHdr->tag.blockNum >= firstDelBlock[i])
+				InvalidateBuffer(bufHdr);	/* releases
spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);

I'm a bit confused about the check here. We hold a buffer partition
lock, and have done a lookup in the mapping table. Why are we then
rechecking the relfilenode/fork/blocknum? And why are we doing so
holding the buffer header lock, which is essentially a spinlock, so
should only ever be held for very short portions?

This looks like it's copying logic from DropRelFileNodeBuffers()
etc, but there the situation is different: We haven't done a buffer
mapping lookup, and we don't hold a partition lock!

That's because the buffer partition lock is released immediately after
the hash table has been looked up. As an aside, InvalidateBuffer()
requires the caller to hold the buffer header spinlock and doesn't hold the

buffer partition lock.

Yes. Holding the buffer header spinlock is necessary to invalidate the buffers.
As for buffer mapping partition lock, as mentioned by Tsunakawa-san, it is
released immediately after BufTableLookup, which is similar to lookup done
in PrefetchSharedBuffer. So I retained these changes.

I have attached the updated patches. Aside from descriptions, no other major
changes in the patch set except 0004. Feedbacks are welcome.

Hi,

Given that I modified the 0004 patch. I repeated the recovery performance
tests I did in [1]/messages/by-id/OSBPR01MB234176B1829AECFE9FDDFCC2EFE90@OSBPR01MB2341.jpnprd01.prod.outlook.com. But this time I used 1000 relations (1MB per relation).
Because of this rel size, it is expected that sequential full buffer scan is
executed for 128MB shared_buffers, while the optimized process is
implemented for the larger shared_buffers.

Below are the results:

[TRUNCATE]
| s_b | MASTER (sec) | PATCHED (sec) |
|--------|--------------|---------------|
| 128 MB | 0.506 | 0.506 |
| 1 GB | 0.906 | 0.506 |
| 20 GB | 19.33 | 0.506 |
| 100 GB | 74.941 | 0.506 |

[VACUUM]
| s_b | MASTER (sec) | PATCHED (sec) |
|--------|--------------|---------------|
| 128 MB | 1.207 | 0.737 |
| 1 GB | 1.707 | 0.806 |
| 20 GB | 14.325 | 0.806 |
| 100 GB | 64.728 | 1.307 |

Looking at the results for both VACUUM and TRUNCATE, we can see
the improvement of performance because of the optimizations.
In addition, there was no regression for the full scan of whole buffer
Pool (as seen in 128MB s_b).

Regards,
Kirk Jamison

[1]: /messages/by-id/OSBPR01MB234176B1829AECFE9FDDFCC2EFE90@OSBPR01MB2341.jpnprd01.prod.outlook.com

#172Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: k.jamison@fujitsu.com (#170)
Re: [Patch] Optimize dropping of relation buffers using dlist

Hello, Kirk. Thank you for the new version.

At Thu, 26 Nov 2020 03:04:10 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in

On Thursday, November 19, 2020 4:08 PM, Tsunakawa, Takayuki wrote:

From: Andres Freund <andres@anarazel.de>

DropRelFileNodeBuffers() in recovery? The most common path is
DropRelationFiles()->smgrdounlinkall()->DropRelFileNodesAllBuffers(),
which 3/4 doesn't address and 4/4 doesn't mention.

4/4 seems to address DropRelationFiles(), but only talks about

TRUNCATE?

Yes. DropRelationFiles() is used in the following two paths:

[Replay of TRUNCATE during recovery]
xact_redo_commit/abort() -> DropRelationFiles() -> smgrdounlinkall() ->
DropRelFileNodesAllBuffers()

[COMMIT/ROLLBACK PREPARED]
FinishPreparedTransaction() -> DropRelationFiles() -> smgrdounlinkall()
-> DropRelFileNodesAllBuffers()

Yes. The concern is that it was not clear in the function descriptions and commit logs
what the optimizations for the DropRelFileNodeBuffers() and DropRelFileNodesAllBuffers()
are for. So I revised only the function description of DropRelFileNodeBuffers() and the
commit logs of the 0003-0004 patches. Please check if the brief descriptions would suffice.

I read the commit message of 3/4. (Though this is not involved
literally in the final commit.)

While recovery, when WAL files of XLOG_SMGR_TRUNCATE from vacuum
or autovacuum are replayed, the buffers are dropped when the sizes
of all involved forks of a relation are already "cached". We can get

This sentence seems missing "dropped by (or using) what".

a reliable size of nblocks for supplied relation's fork at that time,
and it's safe because DropRelFileNodeBuffers() relies on the behavior
that cached nblocks will not be invalidated by file extension during
recovery. Otherwise, or if not in recovery, proceed to sequential
search of the whole buffer pool.

This sentence seems involving confusion. It reads as if "we can rely
on it because we're relying on it". And "the cached value won't be
invalidated" doesn't explain the reason precisely. The reason I think
is that the cached value is guaranteed to be the maximum page we have
in shared buffer at least while recovery, and that guarantee is holded
by not asking fseek once we cached the value.

I also don't get why 4/4 would be a good idea on its own. It uses
BUF_DROP_FULL_SCAN_THRESHOLD to guard
FindAndDropRelFileNodeBuffers() on a per relation basis. But since
DropRelFileNodesAllBuffers() can be used for many relations at once,
this could end up doing BUF_DROP_FULL_SCAN_THRESHOLD - 1

lookups a lot

of times, once for each of nnodes relations?

So, the threshold value should be compared with the total number of blocks
of all target relations, not each relation. You seem to be right, got it.

Fixed this in 0004 patch. Now we compare the total number of buffers-to-be-invalidated
For ALL relations to the BUF_DROP_FULL_SCAN_THRESHOLD.

I didn't see the previous version, but the row of additional
palloc/pfree's in this version looks uneasy.

 	int			i,
+				j,
+				*nforks,
 				n = 0;

Perhaps I think we don't define variable in different types at once.
(I'm not sure about defining multple variables at once.)

@@ -3110,7 +3125,10 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
DropRelFileNodeAllLocalBuffers(rnodes[i].node);
}
else
+ {
+ rels[n] = smgr_reln[i];
nodes[n++] = rnodes[i].node;
+ }
}

We don't need to remember nodes and rnodes here since rnodes[n] is
rels[n]->smgr_rnode here. Or we don't even need to store rels since
we can scan the smgr_reln later again.

nodes is needed in the full-scan path but it is enough to collect it
after finding that we do full-scan.

 	/*
@@ -3120,6 +3138,68 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	if (n == 0)
 	{
 		pfree(nodes);
+		pfree(rels);
+		pfree(rnodes);
+		return;
+	}
+
+	nforks = palloc(sizeof(int) * n);
+	forks = palloc(sizeof(ForkNumber *) * n);
+	blocks = palloc(sizeof(BlockNumber *) * n);
+	firstDelBlocks = palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));
+	for (i = 0; i < n; i++)
+	{
+		forks[i] = palloc(sizeof(ForkNumber) * (MAX_FORKNUM + 1));
+		blocks[i] = palloc(sizeof(BlockNumber) * (MAX_FORKNUM + 1));
+	}

We can allocate the whole array at once like this.

BlockNumber (*blocks)[MAX_FORKNUM+1] =
(BlockNumber (*)[MAX_FORKNUM+1])
palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1))

The elements of forks[][] and blocks[][] are not initialized bacause
some of the elemets may be skipped due to the absense of the
corresponding fork.

+			if (!smgrexists(rels[i], j))
+				continue;
+
+			/* Get the number of blocks for a relation's fork */
+			blocks[i][numForks] = smgrnblocks(rels[i], j, NULL);

If we see a fork which its size is not cached we must give up this
optimization for all target relations.

+			nBlocksToInvalidate += blocks[i][numForks];
+
+			forks[i][numForks++] = j;

We can signal to the later code the absense of a fork by setting
InvalidBlockNumber to blocks. Thus forks[], nforks and numForks can be
removed.

+	/* Zero the array of blocks because these will all be dropped anyway */
+	MemSet(firstDelBlocks, 0, sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));

We don't need to prepare nforks, forks and firstDelBlocks for all
relations before looping over relations. In other words, we can fill
in the arrays for a relation at every iteration of relations.

+ * We enter the optimization iff we are in recovery and the number of blocks to

This comment ticks out of 80 columns. (I'm not sure whether that
convention is still valid..)

+ if (InRecovery && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)

We don't need to check InRecovery here. DropRelFileNodeBuffers doesn't
do that.

+		for (j = 0; j < n; j++)
+		{
+			FindAndDropRelFileNodeBuffers(nodes[j], forks[j], nforks[j],

i is not used at this nesting level so we can use i here.

Also, how is 4/4 safe - this is outside of recovery too?

It seems that DropRelFileNodesAllBuffers() should trigger the new
optimization path only when InRecovery == true, because it intentionally
doesn't check the "accurate" value returned from smgrnblocks().

Fixed it in 0004 patch. Now we ensure that we only enter the optimization path
Iff during recovery.

If the size of any of the target relations is not cached, we give up
the optimization at all even while recoverying. Or am I missing
something?

From: Amit Kapila <amit.kapila16@gmail.com>
On Wed, Nov 18, 2020 at 11:43 PM Andres Freund <andres@anarazel.de>

I'm also worried about the cases where this could cause buffers left
in the buffer pool, without a crosscheck like Thomas' patch would
allow to add. Obviously other processes can dirty buffers in
hot_standby, so any leftover buffer could have bad consequences.

The problem can only arise if other processes extend the relation. The idea
was that in recovery it extends relation by one process which helps to
maintain the cache. Kirk seems to have done testing to cross-verify it by using
his first patch (Prevent-invalidating-blocks-in-smgrextend-during). Which
other crosscheck you are referring here?

I agree that we can do a better job by expanding comments to clearly state
why it is safe.

Yes, basically what Amit-san also mentioned above. The first patch prevents that.
And in the description of DropRelFileNodeBuffers in the 0003 patch, please check
If that would suffice.

+ *		While in recovery, if the expected maximum number of buffers to be
+ *		dropped is small enough and the sizes of all involved forks are
+ *		already cached, individual buffer is located by BufTableLookup().
+ *		It is safe because cached blocks will not be invalidated by file
+ *		extension during recovery.  See smgrnblocks() and smgrextend() for
+ *		more details. Otherwise, if the conditions for optimization are not
+ *		met, the buffer pool is sequentially scanned so that no buffers are
+ *		left behind.

I'm not confident on it, but it seems somewhat obscure. How about
something like this?

We mustn't leave a buffer for the relations to be dropped. We
invalidate buffer blocks by locating using BufTableLookup() when we
assure that we know up to what page of every fork we possiblly have a
buffer for. We can know that by the "cached" flag returned by
smgrblocks. It currently gets true only while recovery. See
smgrnblocks() and smgrextend(). Otherwise we scan the whole buffer
pool to find buffers for the relation, which is slower when a small
part of buffers are to be dropped.

Smaller comment:

+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber

*forkNum,

int nforks,
+							  BlockNumber
*nForkBlocks, BlockNumber *firstDelBlock) ...
+			/* Check that it is in the buffer pool. If not, do

nothing.

*/
+			LWLockAcquire(bufPartitionLock, LW_SHARED);
+			buf_id = BufTableLookup(&bufTag, bufHash);
...
+			bufHdr = GetBufferDescriptor(buf_id);
+
+			buf_state = LockBufHdr(bufHdr);
+
+			if (RelFileNodeEquals(bufHdr->tag.rnode, rnode)

&&

+				bufHdr->tag.forkNum == forkNum[i] &&
+				bufHdr->tag.blockNum >= firstDelBlock[i])
+				InvalidateBuffer(bufHdr);	/* releases
spinlock */
+			else
+				UnlockBufHdr(bufHdr, buf_state);

I'm a bit confused about the check here. We hold a buffer partition
lock, and have done a lookup in the mapping table. Why are we then
rechecking the relfilenode/fork/blocknum? And why are we doing so
holding the buffer header lock, which is essentially a spinlock, so
should only ever be held for very short portions?

This looks like it's copying logic from DropRelFileNodeBuffers() etc,
but there the situation is different: We haven't done a buffer mapping
lookup, and we don't hold a partition lock!

That's because the buffer partition lock is released immediately after the hash
table has been looked up. As an aside, InvalidateBuffer() requires the caller
to hold the buffer header spinlock and doesn't hold the buffer partition lock.

Yes. Holding the buffer header spinlock is necessary to invalidate the buffers.
As for buffer mapping partition lock, as mentioned by Tsunakawa-san, it is
released immediately after BufTableLookup, which is similar to lookup done in
PrefetchSharedBuffer. So I retained these changes.

I have attached the updated patches. Aside from descriptions, no other major
changes in the patch set except 0004. Feedbacks are welcome.

FWIW, As tunakawa-san mentioned, the partition lock is release
immedately after the look-up. The reason that we may release the
partition lock immediately is that it is OK that the buffer has been
evicted by someone to reuse it for other relations. We can know that
case by rechecking the buffer tag after holding header lock.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#173Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Kyotaro Horiguchi (#172)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 26 Nov 2020 16:18:55 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

+	/* Zero the array of blocks because these will all be dropped anyway */
+	MemSet(firstDelBlocks, 0, sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));

We don't need to prepare nforks, forks and firstDelBlocks for all
relations before looping over relations. In other words, we can fill
in the arrays for a relation at every iteration of relations.

Or even we could call FindAndDropRelFileNodeBuffers() for each
forks. It dones't matter in the performance perspective whether the
function loops over forks or the function is called for each forks.

regrds.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#174k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Kyotaro Horiguchi (#172)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>
Hello, Kirk. Thank you for the new version.

Hi, Horiguchi-san. Thank you for your very helpful feedback.
I'm updating the patches addressing those.

+			if (!smgrexists(rels[i], j))
+				continue;
+
+			/* Get the number of blocks for a relation's fork */
+			blocks[i][numForks] = smgrnblocks(rels[i], j,
NULL);

If we see a fork which its size is not cached we must give up this optimization
for all target relations.

I did not use the "cached" flag in DropRelFileNodesAllBuffers and use InRecovery
when deciding for optimization because of the following reasons:
XLogReadBufferExtended() calls smgrnblocks() to apply changes to relation page
contents. So in DropRelFileNodeBuffers(), XLogReadBufferExtended() is called
during VACUUM replay because VACUUM changes the page content.
OTOH, TRUNCATE doesn't change the relation content, it just truncates relation pages
without changing the page contents. So XLogReadBufferExtended() is not called, and
the "cached" flag will always return false. I tested with "cached" flags before, and this
always return false, at least in DropRelFileNodesAllBuffers. Due to this, we cannot use
the cached flag in DropRelFileNodesAllBuffers(). However, I think we can still rely on
smgrnblocks to get the file size as long as we're InRecovery. That cached nblocks is still
guaranteed to be the maximum in the shared buffer.
Thoughts?

Regards,
Kirk Jamison

#175Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: k.jamison@fujitsu.com (#174)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Fri, 27 Nov 2020 02:19:57 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>
Hello, Kirk. Thank you for the new version.

Hi, Horiguchi-san. Thank you for your very helpful feedback.
I'm updating the patches addressing those.

+			if (!smgrexists(rels[i], j))
+				continue;
+
+			/* Get the number of blocks for a relation's fork */
+			blocks[i][numForks] = smgrnblocks(rels[i], j,
NULL);

If we see a fork which its size is not cached we must give up this optimization
for all target relations.

I did not use the "cached" flag in DropRelFileNodesAllBuffers and use InRecovery
when deciding for optimization because of the following reasons:
XLogReadBufferExtended() calls smgrnblocks() to apply changes to relation page
contents. So in DropRelFileNodeBuffers(), XLogReadBufferExtended() is called
during VACUUM replay because VACUUM changes the page content.
OTOH, TRUNCATE doesn't change the relation content, it just truncates relation pages
without changing the page contents. So XLogReadBufferExtended() is not called, and
the "cached" flag will always return false. I tested with "cached" flags before, and this

A bit different from the point, but if some tuples have been inserted
to the truncated table, XLogReadBufferExtended() is called for the
table and the length is cached.

always return false, at least in DropRelFileNodesAllBuffers. Due to this, we cannot use
the cached flag in DropRelFileNodesAllBuffers(). However, I think we can still rely on
smgrnblocks to get the file size as long as we're InRecovery. That cached nblocks is still
guaranteed to be the maximum in the shared buffer.
Thoughts?

That means that we always think as if smgrnblocks returns "cached" (or
"safe") value during recovery, which is out of our current
consensus. If we go on that side, we don't need to consult the
"cached" returned from smgrnblocks at all and it's enough to see only
InRecovery.

I got confused..

We are relying on the "fact" that the first lseek() call of a
(startup) process tells the truth. We added an assertion so that we
make sure that the cached value won't be cleared during recovery. A
possible remaining danger would be closing of an smgr object of a live
relation just after a file extension failure. I think we are thinking
that that doesn't happen during recovery. Although it seems to me
true, I'm not confident.

If that's true, we don't even need to look at the "cached" flag at all
and always be able to rely on the returned value from msgrnblocks()
during recovery. Otherwise, we need to avoid the danger situation.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#176Alexander Korotkov
aekorotkov@gmail.com
In reply to: Kyotaro Horiguchi (#175)
Re: [Patch] Optimize dropping of relation buffers using dlist

Hi!

I've found this patch is RFC on commitfest application. I've quickly
checked if it's really ready for commit. It seems there are still
unaddressed review notes. I'm going to switch it to WFA.

------
Regards,
Alexander Korotkov

#177tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Kyotaro Horiguchi (#175)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

We are relying on the "fact" that the first lseek() call of a
(startup) process tells the truth. We added an assertion so that we
make sure that the cached value won't be cleared during recovery. A
possible remaining danger would be closing of an smgr object of a live
relation just after a file extension failure. I think we are thinking
that that doesn't happen during recovery. Although it seems to me
true, I'm not confident.

If that's true, we don't even need to look at the "cached" flag at all
and always be able to rely on the returned value from msgrnblocks()
during recovery. Otherwise, we need to avoid the danger situation.

Hmm, I've gotten to think that smgrnblocks() doesn't need the cached parameter, too. DropRel*Buffers() can just check InRecovery. Regarding the only concern about smgrclose() by startup process, I was afraid of the cache invalidation by CacheInvalidateSmgr(), but startup process doesn't receive shared inval messages. So, it doesn't call smgrclose*() due to shared cache invalidation.

[InitRecoveryTransactionEnvironment()]
/* Initialize shared invalidation management for Startup process, being
* Initialize shared invalidation management for Startup process, being
* careful to register ourselves as a sendOnly process so we don't need to
* read messages, nor will we get signaled when the queue starts filling
* up.
*/
SharedInvalBackendInit(true);

Kirk-san,
Can you check to see if smgrclose() and its friends are not called during recovery using the regression test?

Regards
Takayuki Tsunakawa

#178k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Kyotaro Horiguchi (#172)
4 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, November 26, 2020 4:19 PM, Horiguchi-san wrote:

Hello, Kirk. Thank you for the new version.

Apologies for the delay, but attached are the updated versions to simplify the patches.
The changes reflected most of your comments/suggestions.

Summary of changes in the latest versions.
1. Updated the function description of DropRelFileNodeBuffers in 0003.
2. Updated the commit logs of 0003 and 0004.
3, FindAndDropRelFileNodeBuffers is now called for each relation fork,
instead of for all involved forks.
4. Removed the unnecessary palloc() and subscripts like forks[][],
firstDelBlock[], nforks, as advised by Horiguchi-san. The memory
allocation for block[][] was also simplified.
So 0004 became simpler and more readable.

At Thu, 26 Nov 2020 03:04:10 +0000, "k.jamison@fujitsu.com"
<k.jamison@fujitsu.com> wrote in

On Thursday, November 19, 2020 4:08 PM, Tsunakawa, Takayuki wrote:

From: Andres Freund <andres@anarazel.de>

DropRelFileNodeBuffers() in recovery? The most common path is
DropRelationFiles()->smgrdounlinkall()->DropRelFileNodesAllBuffers
(), which 3/4 doesn't address and 4/4 doesn't mention.

4/4 seems to address DropRelationFiles(), but only talks about

TRUNCATE?

Yes. DropRelationFiles() is used in the following two paths:

[Replay of TRUNCATE during recovery]
xact_redo_commit/abort() -> DropRelationFiles() ->
smgrdounlinkall() ->
DropRelFileNodesAllBuffers()

[COMMIT/ROLLBACK PREPARED]
FinishPreparedTransaction() -> DropRelationFiles() ->
smgrdounlinkall()
-> DropRelFileNodesAllBuffers()

Yes. The concern is that it was not clear in the function descriptions
and commit logs what the optimizations for the
DropRelFileNodeBuffers() and DropRelFileNodesAllBuffers() are for. So
I revised only the function description of DropRelFileNodeBuffers() and the

commit logs of the 0003-0004 patches. Please check if the brief descriptions
would suffice.

I read the commit message of 3/4. (Though this is not involved literally in the
final commit.)

While recovery, when WAL files of XLOG_SMGR_TRUNCATE from vacuum

or

autovacuum are replayed, the buffers are dropped when the sizes of all
involved forks of a relation are already "cached". We can get

This sentence seems missing "dropped by (or using) what".

a reliable size of nblocks for supplied relation's fork at that time,
and it's safe because DropRelFileNodeBuffers() relies on the behavior
that cached nblocks will not be invalidated by file extension during
recovery. Otherwise, or if not in recovery, proceed to sequential
search of the whole buffer pool.

This sentence seems involving confusion. It reads as if "we can rely on it
because we're relying on it". And "the cached value won't be invalidated"
doesn't explain the reason precisely. The reason I think is that the cached
value is guaranteed to be the maximum page we have in shared buffer at least
while recovery, and that guarantee is holded by not asking fseek once we
cached the value.

Fixed the commit log of 0003.

I also don't get why 4/4 would be a good idea on its own. It uses
BUF_DROP_FULL_SCAN_THRESHOLD to guard
FindAndDropRelFileNodeBuffers() on a per relation basis. But since
DropRelFileNodesAllBuffers() can be used for many relations at
once, this could end up doing BUF_DROP_FULL_SCAN_THRESHOLD

- 1

lookups a lot

of times, once for each of nnodes relations?

So, the threshold value should be compared with the total number of
blocks of all target relations, not each relation. You seem to be right, got

it.

Fixed this in 0004 patch. Now we compare the total number of
buffers-to-be-invalidated For ALL relations to the

BUF_DROP_FULL_SCAN_THRESHOLD.

I didn't see the previous version, but the row of additional palloc/pfree's in
this version looks uneasy.

Fixed this too.

int			i,
+				j,
+				*nforks,
n = 0;

Perhaps I think we don't define variable in different types at once.
(I'm not sure about defining multple variables at once.)

Fixed this too.

@@ -3110,7 +3125,10 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend
*rnodes, int nnodes)

DropRelFileNodeAllLocalBuffers(rnodes[i].node);
}
else
+ {
+ rels[n] = smgr_reln[i];
nodes[n++] = rnodes[i].node;
+ }
}

We don't need to remember nodes and rnodes here since rnodes[n] is
rels[n]->smgr_rnode here. Or we don't even need to store rels since we can
scan the smgr_reln later again.

nodes is needed in the full-scan path but it is enough to collect it after finding
that we do full-scan.

I followed your advice and removed the rnodes[] and rels[].
nodes[] is allocated later at full scan path.

+	nforks = palloc(sizeof(int) * n);
+	forks = palloc(sizeof(ForkNumber *) * n);
+	blocks = palloc(sizeof(BlockNumber *) * n);
+	firstDelBlocks = palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM
+ 1));
+	for (i = 0; i < n; i++)
+	{
+		forks[i] = palloc(sizeof(ForkNumber) * (MAX_FORKNUM +
1));
+		blocks[i] = palloc(sizeof(BlockNumber) * (MAX_FORKNUM
+ 1));
+	}

We can allocate the whole array at once like this.

BlockNumber (*blocks)[MAX_FORKNUM+1] =
(BlockNumber (*)[MAX_FORKNUM+1])
palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1))

Thank you for suggesting to reduce the lines for the 2d dynamic memory alloc.
I followed this way in 0004, but it's my first time to see it written this way.
I am very glad it works, though is it okay to write it this way since I cannot find
a similar code of declaring and allocating 2D arrays like this in Postgres source code?

+			nBlocksToInvalidate += blocks[i][numForks];
+
+			forks[i][numForks++] = j;

We can signal to the later code the absense of a fork by setting
InvalidBlockNumber to blocks. Thus forks[], nforks and numForks can be
removed.

Followed it in 0004.

+	/* Zero the array of blocks because these will all be dropped anyway
*/
+	MemSet(firstDelBlocks, 0, sizeof(BlockNumber) * n *
(MAX_FORKNUM +
+1));

We don't need to prepare nforks, forks and firstDelBlocks for all relations
before looping over relations. In other words, we can fill in the arrays for a
relation at every iteration of relations.

Followed your advice. Although I now drop the buffers per fork, which now
removes forks[][], nforks, firstDelBlocks[].

+	 * We enter the optimization iff we are in recovery and the number of
+blocks to

This comment ticks out of 80 columns. (I'm not sure whether that convention
is still valid..)

Fixed.

+ if (InRecovery && nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD)

We don't need to check InRecovery here. DropRelFileNodeBuffers doesn't do
that.

As for DropRelFileNodesAllBuffers use case, I used InRecovery
so that the optimization still works.
Horiguchi-san also wrote in another mail:

A bit different from the point, but if some tuples have been inserted to the
truncated table, XLogReadBufferExtended() is called for the table and the
length is cached.

I was wrong in my previous claim that the "cached" value always return false.
When I checked the recovery test log from recovery tap test, there was only
one example when "cached" became true (script below) and entered the
optimization path. However, in all other cases including the TRUNCATE test case
in my patch, the "cached" flag returns "false".

"cached" flag became true:
# in different subtransaction patterns
$node->safe_psql(
'postgres', "
BEGIN;
CREATE TABLE spc_commit (id serial PRIMARY KEY, id2 int);
INSERT INTO spc_commit VALUES (DEFAULT, generate_series(1,3000));
TRUNCATE spc_commit;
SAVEPOINT s; ALTER TABLE spc_commit SET TABLESPACE other; RELEASE s;
COPY spc_commit FROM '$copy_file' DELIMITER ',';
COMMIT;");
$node->stop('immediate');
$node->start;

So I used the InRecovery for the optimization case of DropRelFileNodesAllBuffers.
I retained the smgrnblocks' "cached" parameter as it is useful in
DropRelFileNodeBuffers.

I agree that we can do a better job by expanding comments to clearly
state why it is safe.

Yes, basically what Amit-san also mentioned above. The first patch

prevents that.

And in the description of DropRelFileNodeBuffers in the 0003 patch,
please check If that would suffice.

+ *		While in recovery, if the expected maximum number of
buffers to be
+ *		dropped is small enough and the sizes of all involved forks
are
+ *		already cached, individual buffer is located by
BufTableLookup().
+ *		It is safe because cached blocks will not be invalidated by file
+ *		extension during recovery.  See smgrnblocks() and
smgrextend() for
+ *		more details. Otherwise, if the conditions for optimization are
not
+ *		met, the buffer pool is sequentially scanned so that no
buffers are
+ *		left behind.

I'm not confident on it, but it seems somewhat obscure. How about
something like this?

We mustn't leave a buffer for the relations to be dropped. We invalidate
buffer blocks by locating using BufTableLookup() when we assure that we
know up to what page of every fork we possiblly have a buffer for. We can
know that by the "cached" flag returned by smgrblocks. It currently gets true
only while recovery. See
smgrnblocks() and smgrextend(). Otherwise we scan the whole buffer pool to
find buffers for the relation, which is slower when a small part of buffers are
to be dropped.

Followed your advice and modified it a bit.

I have changed the status to "Needs Review".
Feedbacks are always welcome.

Regards,
Kirk Jamison

Attachments:

v33-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v33-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From 13bd93d5f20ae33b3ba028df32f5a18c90403945 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v33 1/4] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v33-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v33-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From b2a7e57956a8cc8bcac73b6f5caec47eec261657 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v33 2/4] Add bool param in smgrnblocks() for cached blocks.

The flag ensures that we return a reliable value from smgrnblocks.
---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 21 ++++++++++++++++++---
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 34 insertions(+), 18 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 32a3099..afc640d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index d538f257..3874ff3 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -721,7 +721,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ad0d1a9..1680bf4 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -740,7 +740,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2857,7 +2857,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..e9dffd2 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -535,7 +535,6 @@ smgrwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 										buffer, skipFsync);
 }
 
-
 /*
  *	smgrwriteback() -- Trigger kernel writeback for the supplied range of
  *					   blocks.
@@ -551,18 +550,34 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 /*
  *	smgrnblocks() -- Calculate the number of blocks in the
  *					 supplied relation.
+ *
+ *		The "cached" flag ensures that no buffers exist for blocks after the
+ *		cached value is known to the process during recovery.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *cached)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  In recovery, the cached
+	 * value returned by the first lseek could be smaller than the actual number
+	 * of existing buffers of the file, which is caused by buggy Linux kernels
+	 * that might not have accounted for the recent write.  However, we can still
+	 * rely on the cached value even if we get a bogus value from first lseek
+	 * since it is impossible to have buffer for blocks after that file size.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (cached != NULL)
+			*cached = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
+
+	if (cached != NULL)
+		*cached = false;
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..cd99f1b 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *cached);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v33-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v33-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 926f97c8ffc9cfe3237a44ec47854d5d3f7bc4a9 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Tue, 10 Nov 2020 06:27:11 +0000
Subject: [PATCH v33 3/4] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning. This improves the
performance especially when VACUUM or autovacuum truncated off any of
the empty pages at the end of relation.

While recovery, when WAL files of XLOG_SMGR_TRUNCATE from vacuum or
autovacuum are replayed, we must not leave a buffer for the relations
to be dropped.  So we invalidate buffer blocks by locating using
BufTableLookup() when it is certain that we know up to what page of
every fork we possiblly have a buffer.  We can know that by the
"cached" flag returned by smgrnblocks(), which currently gets true
only while recovery.  It's safe to use the cached nblocks because it
is guaranteed to be the maximum page we have in shared buffer during
recovery, and that guarantee is held by not asking fseek once we have
cached the value.  Also, the cached nblocks will not be invalidated
by file extension during recovery. See smgrnblocks() and smgrextend()
for more details.

If we are not in recovery or when the nblocks is not cached, then we
scan the whole buffer pool to search and drop the buffers of relation,
which is slower when a small part of buffers are to be dropped.
---
 src/backend/storage/buffer/bufmgr.c | 115 +++++++++++++++++++++++++++++++++---
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 110 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 1680bf4..c3ee6c6 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 256)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,10 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FindAndDropRelFileNodeBuffers(RelFileNode rnode,
+										  ForkNumber forkNum,
+										  BlockNumber nForkBlock,
+										  BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2967,18 +2973,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		We must not leave a buffer for the relations to be dropped.  We
+ *		invalidate buffer blocks by locating using BufTableLookup() when it
+ *		is certain that we know up to what page of every fork we possiblly
+ *		have a buffer.  We can know that by the "cached" flag returned by
+ *		smgrnblocks(), which currently gets true only while recovery.  See
+ *		smgrnblocks() and smgrextend().  Otherwise we scan the whole buffer
+ *		pool to find buffers for the relation, which is slower when a small
+ *		part of buffers are to be dropped.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		cached = false;
+	BlockNumber	nForkBlock[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2992,6 +3008,38 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of to-be-invalidated blocks of a relation as
+	 * well as the total blocks for a given fork.  Give up the optimization
+	 * if the block is not cached.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlock[i] = smgrnblocks(smgr_reln, forkNum[i], &cached);
+
+		if (!cached)
+			break;
+
+		/* Get the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlock[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffers in the hashtable and drop them if the block size
+	 * is already cached and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			FindAndDropRelFileNodeBuffers(rnode.node, forkNum[j],
+										  nForkBlock[j], firstDelBlock[j]);
+		}
+		return;
+	}
+
 	for (i = 0; i < NBuffers; i++)
 	{
 		BufferDesc *bufHdr = GetBufferDescriptor(i);
@@ -3135,6 +3183,60 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 }
 
 /* ---------------------------------------------------------------------
+ *		FindAndDropRelFileNodeBuffers
+ *
+ *		This function finds and removes from the buffer pool all the pages
+ *		of the specified relation fork that has block number >= firstDelBlock.
+ *		(In particular, with firstDelBlock = 0, all pages are removed.)
+ *		This is only called in recovery when the block count of a fork is
+ *		cached or when the total number of to-be-invalidated blocks for
+ *		involved relation/s does not exceed the threshold for buffer full scan.
+ * --------------------------------------------------------------------
+ */
+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber forkNum,
+							  BlockNumber nForkBlock, BlockNumber firstDelBlock)
+{
+	BlockNumber		curBlock;
+
+	for (curBlock = firstDelBlock; curBlock < nForkBlock; curBlock++)
+	{
+		uint32		bufHash;		/* hash value for tag */
+		BufferTag	bufTag;			/* identity of requested block */
+		LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+		int		buf_id;
+		BufferDesc	*bufHdr;
+		uint32		buf_state;
+
+		/* create a tag so we can lookup the buffer */
+		INIT_BUFFERTAG(bufTag, rnode, forkNum, curBlock);
+
+		/* determine its hash code and partition lock ID */
+		bufHash = BufTableHashCode(&bufTag);
+		bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+		/* Check that it is in the buffer pool. If not, do nothing. */
+		LWLockAcquire(bufPartitionLock, LW_SHARED);
+		buf_id = BufTableLookup(&bufTag, bufHash);
+		LWLockRelease(bufPartitionLock);
+
+		if (buf_id < 0)
+			continue;
+
+		bufHdr = GetBufferDescriptor(buf_id);
+
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+/* ---------------------------------------------------------------------
  *		DropDatabaseBuffers
  *
  *		This function removes all the buffers in the buffer cache for a
@@ -3246,8 +3348,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index e9dffd2..9d3a67c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -605,7 +605,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v33-0004-Optimize-DropRelFileNodesAllBuffers-in-recovery.patchapplication/octet-stream; name=v33-0004-Optimize-DropRelFileNodesAllBuffers-in-recovery.patchDownload
From 4bb6343e231fdcb28b0b71dda9ed8f6550dbb7bd Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 2 Dec 2020 12:48:04 +0000
Subject: [PATCH v33 4/4] Optimize DropRelFileNodesAllBuffers() in recovery.

DropRelFileNodesAllBuffers() is optimized to skip the time-consuming
scan of the whole buffer pool during recovery when the relation is
small enough, or when the number of blocks to be invalidated is below
the full scan threshold. This improves the DropRelationFiles()
performance when the TRUNCATE command truncated off any of the empty
pages at the end of relation, and when dropping relation buffers if a
commit/rollback transaction has been prepared in FinishPreparedTransaction().
---
 src/backend/storage/buffer/bufmgr.c | 85 ++++++++++++++++++++++++++++++++-----
 src/backend/storage/smgr/smgr.c     | 14 +++---
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 83 insertions(+), 18 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index c3ee6c6..8319208 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3091,28 +3091,27 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
+DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 {
-	int			i,
-				n = 0;
+	int			i;
+	int			j;
+	int			n = 0;
 	RelFileNode *nodes;
 	bool		use_bsearch;
 
 	if (nnodes == 0)
 		return;
 
-	nodes = palloc(sizeof(RelFileNode) * nnodes);	/* non-local relations */
-
 	/* If it's a local relation, it's localbuf.c's problem. */
 	for (i = 0; i < nnodes; i++)
 	{
-		if (RelFileNodeBackendIsTemp(rnodes[i]))
+		if (RelFileNodeBackendIsTemp(smgr_reln[i]->smgr_rnode))
 		{
-			if (rnodes[i].backend == MyBackendId)
-				DropRelFileNodeAllLocalBuffers(rnodes[i].node);
+			if (smgr_reln[i]->smgr_rnode.backend == MyBackendId)
+				DropRelFileNodeAllLocalBuffers(smgr_reln[i]->smgr_rnode.node);
 		}
 		else
-			nodes[n++] = rnodes[i].node;
+			n++;
 	}
 
 	/*
@@ -3120,11 +3119,77 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	 * memory and return.
 	 */
 	if (n == 0)
+		return;
+
+	/*
+	 * We enter the optimization iff we are in recovery.  Otherwise,
+	 * we proceed to full scan of the whole buffer pool.
+	 */
+	if (InRecovery)
 	{
-		pfree(nodes);
+		BlockNumber (*block)[MAX_FORKNUM + 1] = (BlockNumber (*)[MAX_FORKNUM + 1])
+												palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));
+		BlockNumber	nBlocksToInvalidate = 0;
+
+		for (i = 0; i < n; i++)
+		{
+			for (j = 0; j <= MAX_FORKNUM; j++)
+			{
+				/*
+				 * Assign block to InvalidblockNumber if a relation
+				 * fork does not exist, so that we can skip it later
+				 * when dropping the relation buffers.
+				 */
+				if (!smgrexists(smgr_reln[i], j))
+				{
+					block[i][j] = InvalidBlockNumber;
+					continue;
+				}
+
+				/* Get the number of blocks for a relation's fork */
+				block[i][j] = smgrnblocks(smgr_reln[i], j, NULL);
+
+				nBlocksToInvalidate += block[i][j];
+			}
+		}
+
+		/*
+		 * Give up the optimization if the total number of blocks
+		 * to be invalidated for all relations is greater than or
+		 * equal to the full scan threshold.
+		 */
+		if (nBlocksToInvalidate >= BUF_DROP_FULL_SCAN_THRESHOLD)
+		{
+			pfree(block);
+			goto buffer_full_scan;
+		}
+
+		for (i = 0; i < n; i++)
+		{
+			/*
+			 * If block to drop is valid, drop the buffers of the fork.
+			 * Zero the firstDelBlock because all buffers will be
+			 * dropped anyway.
+			 */
+			for (j = 0; j <= MAX_FORKNUM; j++)
+			{
+				if (BlockNumberIsValid(block[i][j]))
+				{
+					FindAndDropRelFileNodeBuffers(smgr_reln[i]->smgr_rnode.node,
+												  j, block[i][j], 0);
+				}
+			}
+		}
+
+		pfree(block);
 		return;
 	}
 
+buffer_full_scan:
+	nodes = palloc(sizeof(RelFileNode) * n); /* non-local relations */
+	for (i = 0; i < n; i++)
+		nodes[i] = smgr_reln[i]->smgr_rnode.node;
+
 	/*
 	 * For low number of relations to drop just use a simple walk through, to
 	 * save the bsearch overhead. The threshold to use is rather a guess than
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 9d3a67c..3663bb7 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -392,7 +392,13 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 		return;
 
 	/*
-	 * create an array which contains all relations to be dropped, and close
+	 * Get rid of any remaining buffers for the relations.  bufmgr will just
+	 * drop them without bothering to write the contents.
+	 */
+	DropRelFileNodesAllBuffers(rels, nrels);
+
+	/*
+	 * Create an array which contains all relations to be dropped, and close
 	 * each relation's forks at the smgr level while at it
 	 */
 	rnodes = palloc(sizeof(RelFileNodeBackend) * nrels);
@@ -409,12 +415,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	}
 
 	/*
-	 * Get rid of any remaining buffers for the relations.  bufmgr will just
-	 * drop them without bothering to write the contents.
-	 */
-	DropRelFileNodesAllBuffers(rnodes, nrels);
-
-	/*
 	 * It'd be nice to tell the stats collector to forget them immediately,
 	 * too. But we can't because we don't know the OIDs.
 	 */
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 056f65e..2e5189b 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -205,7 +205,7 @@ extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels)
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
-extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
+extern void DropRelFileNodesAllBuffers(struct SMgrRelationData **smgr_reln, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
 #define RelationGetNumberOfBlocks(reln) \
-- 
1.8.3.1

#179tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#178)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Apologies for the delay, but attached are the updated versions to simplify the
patches.

Looks good for me. Thanks to Horiguchi-san and Andres-san, the code bebecame further compact and easier to read. I've marked this ready for committer.

To the committer:
I don't think it's necessary to refer to COMMIT/ROLLBACK PREPARED in the following part of the 0003 commit message. They surely call DropRelFileNodesAllBuffers(), but COMMIT/ROLLBACK also call it.

the full scan threshold. This improves the DropRelationFiles()
performance when the TRUNCATE command truncated off any of the empty
pages at the end of relation, and when dropping relation buffers if a
commit/rollback transaction has been prepared in FinishPreparedTransaction().

Regards
Takayuki Tsunakawa

#180Tang, Haiying
tanghy.fnst@cn.fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#179)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hello, Kirk

Thanks for providing the new patches.
I did the recovery performance test on them, the results look good. I'd like to share them with you and everyone else.
(I also record VACUUM and TRUNCATE execution time on master/primary in case you want to have a look.)

1. VACUUM and Failover test results(average of 15 times)
[VACUUM] ---execution time on master/primary
shared_buffers master(sec) patched(sec) %reg=((patched-master)/master)
--------------------------------------------------------------------------------------
128M 9.440 9.483 0%
10G 74.689 76.219 2%
20G 152.538 138.292 -9%

[Failover] ---execution time on standby
shared_buffers master(sec) patched(sec) %reg=((patched-master)/master)
--------------------------------------------------------------------------------------
128M 3.629 2.961 -18%
10G 82.443 2.627 -97%
20G 171.388 2.607 -98%

2. TRUNCATE and Failover test results(average of 15 times)
[TRUNCATE] ---execution time on master/primary
shared_buffers master(sec) patched(sec) %reg=((patched-master)/master)
--------------------------------------------------------------------------------------
128M 49.271 49.867 1%
10G 172.437 175.197 2%
20G 279.658 278.752 0%

[Failover] ---execution time on standby
shared_buffers master(sec) patched(sec) %reg=((patched-master)/master)
--------------------------------------------------------------------------------------
128M 4.877 3.989 -18%
10G 92.680 3.975 -96%
20G 182.035 3.962 -98%

[Machine spec]
CPU : 40 processors (Intel(R) Xeon(R) Silver 4210 CPU @ 2.20GHz)
Memory: 64G
OS: CentOS 8

[Failover test data]
Total table Size: 700M
Table: 10000 tables (1000 rows per table)

If you have question on my test, please let me know.

Regards,
Tang

#181Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: k.jamison@fujitsu.com (#178)
Re: [Patch] Optimize dropping of relation buffers using dlist

Thanks for the new version.

This contains only replies. I'll send some further comments in another
mail later.

At Thu, 3 Dec 2020 03:49:27 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in

On Thursday, November 26, 2020 4:19 PM, Horiguchi-san wrote:

Hello, Kirk. Thank you for the new version.

Apologies for the delay, but attached are the updated versions to simplify the patches.
The changes reflected most of your comments/suggestions.

Summary of changes in the latest versions.
1. Updated the function description of DropRelFileNodeBuffers in 0003.
2. Updated the commit logs of 0003 and 0004.
3, FindAndDropRelFileNodeBuffers is now called for each relation fork,
instead of for all involved forks.
4. Removed the unnecessary palloc() and subscripts like forks[][],
firstDelBlock[], nforks, as advised by Horiguchi-san. The memory
allocation for block[][] was also simplified.
So 0004 became simpler and more readable.

...

a reliable size of nblocks for supplied relation's fork at that time,
and it's safe because DropRelFileNodeBuffers() relies on the behavior
that cached nblocks will not be invalidated by file extension during
recovery. Otherwise, or if not in recovery, proceed to sequential
search of the whole buffer pool.

This sentence seems involving confusion. It reads as if "we can rely on it
because we're relying on it". And "the cached value won't be invalidated"
doesn't explain the reason precisely. The reason I think is that the cached
value is guaranteed to be the maximum page we have in shared buffer at least
while recovery, and that guarantee is holded by not asking fseek once we
cached the value.

Fixed the commit log of 0003.

Thanks!

...

+	nforks = palloc(sizeof(int) * n);
+	forks = palloc(sizeof(ForkNumber *) * n);
+	blocks = palloc(sizeof(BlockNumber *) * n);
+	firstDelBlocks = palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM
+ 1));
+	for (i = 0; i < n; i++)
+	{
+		forks[i] = palloc(sizeof(ForkNumber) * (MAX_FORKNUM +
1));
+		blocks[i] = palloc(sizeof(BlockNumber) * (MAX_FORKNUM
+ 1));
+	}

We can allocate the whole array at once like this.

BlockNumber (*blocks)[MAX_FORKNUM+1] =
(BlockNumber (*)[MAX_FORKNUM+1])
palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1))

Thank you for suggesting to reduce the lines for the 2d dynamic memory alloc.
I followed this way in 0004, but it's my first time to see it written this way.
I am very glad it works, though is it okay to write it this way since I cannot find
a similar code of declaring and allocating 2D arrays like this in Postgres source code?

Actually it would be somewhat novel for a certain portion of people,
but it is fundamentally the same with function pointers. Hard to make
it from scratch, but I suppose not so hard to read:)

int (*func_char_to_int)(char x) = some_func;

FWIW isn.c has the following part:

static bool
check_table(const char *(*TABLE)[2], const unsigned TABLE_index[10][2])

+			nBlocksToInvalidate += blocks[i][numForks];
+
+			forks[i][numForks++] = j;

We can signal to the later code the absense of a fork by setting
InvalidBlockNumber to blocks. Thus forks[], nforks and numForks can be
removed.

Followed it in 0004.

Looks fine to me, thanks.

+	/* Zero the array of blocks because these will all be dropped anyway
*/
+	MemSet(firstDelBlocks, 0, sizeof(BlockNumber) * n *
(MAX_FORKNUM +
+1));

We don't need to prepare nforks, forks and firstDelBlocks for all relations
before looping over relations. In other words, we can fill in the arrays for a
relation at every iteration of relations.

Followed your advice. Although I now drop the buffers per fork, which now
removes forks[][], nforks, firstDelBlocks[].

That's fine for me.

+	 * We enter the optimization iff we are in recovery and the number of
+blocks to

This comment ticks out of 80 columns. (I'm not sure whether that convention
is still valid..)

Fixed.

+ if (InRecovery && nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD)

We don't need to check InRecovery here. DropRelFileNodeBuffers doesn't do
that.

As for DropRelFileNodesAllBuffers use case, I used InRecovery
so that the optimization still works.
Horiguchi-san also wrote in another mail:

A bit different from the point, but if some tuples have been inserted to the
truncated table, XLogReadBufferExtended() is called for the table and the
length is cached.

I was wrong in my previous claim that the "cached" value always return false.
When I checked the recovery test log from recovery tap test, there was only
one example when "cached" became true (script below) and entered the
optimization path. However, in all other cases including the TRUNCATE test case
in my patch, the "cached" flag returns "false".

Yeah, I agree that smgrnblocks returns false in the targetted cases,
so we should want some amendment. We need to disucssion on this point.

"cached" flag became true:
# in different subtransaction patterns
$node->safe_psql(
'postgres', "
BEGIN;
CREATE TABLE spc_commit (id serial PRIMARY KEY, id2 int);
INSERT INTO spc_commit VALUES (DEFAULT, generate_series(1,3000));
TRUNCATE spc_commit;
SAVEPOINT s; ALTER TABLE spc_commit SET TABLESPACE other; RELEASE s;
COPY spc_commit FROM '$copy_file' DELIMITER ',';
COMMIT;");
$node->stop('immediate');
$node->start;

So I used the InRecovery for the optimization case of DropRelFileNodesAllBuffers.
I retained the smgrnblocks' "cached" parameter as it is useful in
DropRelFileNodeBuffers.

I think that's ok as this version of the patch.

I agree that we can do a better job by expanding comments to clearly
state why it is safe.

Yes, basically what Amit-san also mentioned above. The first patch

prevents that.

And in the description of DropRelFileNodeBuffers in the 0003 patch,
please check If that would suffice.

+ *		While in recovery, if the expected maximum number of
buffers to be
+ *		dropped is small enough and the sizes of all involved forks
are
+ *		already cached, individual buffer is located by
BufTableLookup().
+ *		It is safe because cached blocks will not be invalidated by file
+ *		extension during recovery.  See smgrnblocks() and
smgrextend() for
+ *		more details. Otherwise, if the conditions for optimization are
not
+ *		met, the buffer pool is sequentially scanned so that no
buffers are
+ *		left behind.

I'm not confident on it, but it seems somewhat obscure. How about
something like this?

We mustn't leave a buffer for the relations to be dropped. We invalidate
buffer blocks by locating using BufTableLookup() when we assure that we
know up to what page of every fork we possiblly have a buffer for. We can
know that by the "cached" flag returned by smgrblocks. It currently gets true
only while recovery. See
smgrnblocks() and smgrextend(). Otherwise we scan the whole buffer pool to
find buffers for the relation, which is slower when a small part of buffers are
to be dropped.

Followed your advice and modified it a bit.

I have changed the status to "Needs Review".
Feedbacks are always welcome.

Regards,
Kirk Jamison

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#182Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#179)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 3 Dec 2020 07:18:16 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Apologies for the delay, but attached are the updated versions to simplify the
patches.

Looks good for me. Thanks to Horiguchi-san and Andres-san, the code bebecame further compact and easier to read. I've marked this ready for committer.

To the committer:
I don't think it's necessary to refer to COMMIT/ROLLBACK PREPARED in the following part of the 0003 commit message. They surely call DropRelFileNodesAllBuffers(), but COMMIT/ROLLBACK also call it.

the full scan threshold. This improves the DropRelationFiles()
performance when the TRUNCATE command truncated off any of the empty
pages at the end of relation, and when dropping relation buffers if a
commit/rollback transaction has been prepared in FinishPreparedTransaction().

I think whether we can use this optimization only by looking
InRecovery is still in doubt. Or if we can decide that on that
criteria, 0003 also can be simplivied using the same assumption.

Separate from the maybe-remaining discussion, I have a comment on the
revised code in 0004.

+		 * equal to the full scan threshold.
+		 */
+		if (nBlocksToInvalidate >= BUF_DROP_FULL_SCAN_THRESHOLD)
+		{
+			pfree(block);
+			goto buffer_full_scan;
+		}

I don't particularily hate goto statement but we can easily avoid that
by reversing the condition here. You might consider the length of the
line calling "FindAndDropRelFileNodeBuffers" but the indentation can
be lowered by inverting the condition on BLockNumberIsValid.

!| if (nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
| {
| for (i = 0; i < n; i++)
| {
| /*
| * If block to drop is valid, drop the buffers of the fork.
| * Zero the firstDelBlock because all buffers will be
| * dropped anyway.
| */
| for (j = 0; j <= MAX_FORKNUM; j++)
| {
!| if (!BlockNumberIsValid(block[i][j]))
!| continue;
|
| FindAndDropRelFileNodeBuffers(smgr_reln[i]->smgr_rnode.node,
| j, block[i][j], 0);
| }
| }
| pfree(block);
| return;
| }
|
| pfree(block);

Or we can separate the calcualtion part and the execution part by
introducing a flag "do_fullscan".

| /*
| * We enter the optimization iff we are in recovery. Otherwise,
| * we proceed to full scan of the whole buffer pool.
| */
| if (InRecovery)
| {
...
!| if (nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
!| do_fullscan = false;
!| }
!|
!| if (!do_fullscan)
!| {
| for (i = 0; i < n; i++)
| {
| /*
| * If block to drop is valid, drop the buffers of the fork.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#183k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Tang, Haiying (#180)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Friday, December 4, 2020 12:42 PM, Tang, Haiying wrote:

Hello, Kirk

Thanks for providing the new patches.
I did the recovery performance test on them, the results look good. I'd like to
share them with you and everyone else.
(I also record VACUUM and TRUNCATE execution time on master/primary in
case you want to have a look.)

Hi, Tang.
Thank you very much for verifying the performance using the latest set of patches.
Although it's not supposed to affect the non-recovery path (execution on primary),
It's good to see those results too.

1. VACUUM and Failover test results(average of 15 times) [VACUUM]
---execution time on master/primary
shared_buffers master(sec)
patched(sec) %reg=((patched-master)/master)
-------------------------------------------------------------------------------------
-
128M 9.440 9.483 0%
10G 74.689 76.219 2%
20G 152.538 138.292 -9%

[Failover] ---execution time on standby
shared_buffers master(sec)
patched(sec) %reg=((patched-master)/master)
-------------------------------------------------------------------------------------
-
128M 3.629 2.961 -18%
10G 82.443 2.627 -97%
20G 171.388 2.607 -98%

2. TRUNCATE and Failover test results(average of 15 times) [TRUNCATE]
---execution time on master/primary
shared_buffers master(sec)
patched(sec) %reg=((patched-master)/master)
-------------------------------------------------------------------------------------
-
128M 49.271 49.867 1%
10G 172.437 175.197 2%
20G 279.658 278.752 0%

[Failover] ---execution time on standby
shared_buffers master(sec)
patched(sec) %reg=((patched-master)/master)
-------------------------------------------------------------------------------------
-
128M 4.877 3.989 -18%
10G 92.680 3.975 -96%
20G 182.035 3.962 -98%

[Machine spec]
CPU : 40 processors (Intel(R) Xeon(R) Silver 4210 CPU @ 2.20GHz)
Memory: 64G
OS: CentOS 8

[Failover test data]
Total table Size: 700M
Table: 10000 tables (1000 rows per table)

If you have question on my test, please let me know.

Looks great.
That was helpful to see if there were any performance differences than the previous
versions' results. But I am glad it turned out great too.

Regards,
Kirk Jamison

#184Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#175)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Nov 27, 2020 at 11:36 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Fri, 27 Nov 2020 02:19:57 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>
Hello, Kirk. Thank you for the new version.

Hi, Horiguchi-san. Thank you for your very helpful feedback.
I'm updating the patches addressing those.

+                   if (!smgrexists(rels[i], j))
+                           continue;
+
+                   /* Get the number of blocks for a relation's fork */
+                   blocks[i][numForks] = smgrnblocks(rels[i], j,
NULL);

If we see a fork which its size is not cached we must give up this optimization
for all target relations.

I did not use the "cached" flag in DropRelFileNodesAllBuffers and use InRecovery
when deciding for optimization because of the following reasons:
XLogReadBufferExtended() calls smgrnblocks() to apply changes to relation page
contents. So in DropRelFileNodeBuffers(), XLogReadBufferExtended() is called
during VACUUM replay because VACUUM changes the page content.
OTOH, TRUNCATE doesn't change the relation content, it just truncates relation pages
without changing the page contents. So XLogReadBufferExtended() is not called, and
the "cached" flag will always return false. I tested with "cached" flags before, and this

A bit different from the point, but if some tuples have been inserted
to the truncated table, XLogReadBufferExtended() is called for the
table and the length is cached.

always return false, at least in DropRelFileNodesAllBuffers. Due to this, we cannot use
the cached flag in DropRelFileNodesAllBuffers(). However, I think we can still rely on
smgrnblocks to get the file size as long as we're InRecovery. That cached nblocks is still
guaranteed to be the maximum in the shared buffer.
Thoughts?

That means that we always think as if smgrnblocks returns "cached" (or
"safe") value during recovery, which is out of our current
consensus. If we go on that side, we don't need to consult the
"cached" returned from smgrnblocks at all and it's enough to see only
InRecovery.

I got confused..

We are relying on the "fact" that the first lseek() call of a
(startup) process tells the truth. We added an assertion so that we
make sure that the cached value won't be cleared during recovery. A
possible remaining danger would be closing of an smgr object of a live
relation just after a file extension failure. I think we are thinking
that that doesn't happen during recovery. Although it seems to me
true, I'm not confident.

Yeah, I also think it might not be worth depending upon whether smgr
close has been done before or not. I feel the current idea of using
'cached' parameter is relatively solid and we should rely on that.
Also, which means that in DropRelFileNodesAllBuffers() we should rely
on the same, I think doing things differently in this regard will lead
to confusion. I agree in some cases we might not get benefits but it
is more important to be correct and keep the code consistent to avoid
introducing bugs now or in the future.

--
With Regards,
Amit Kapila.

#185k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#184)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Friday, December 4, 2020 8:27 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:

On Fri, Nov 27, 2020 at 11:36 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Fri, 27 Nov 2020 02:19:57 +0000, "k.jamison@fujitsu.com"
<k.jamison@fujitsu.com> wrote in

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com> Hello, Kirk.
Thank you for the new version.

Hi, Horiguchi-san. Thank you for your very helpful feedback.
I'm updating the patches addressing those.

+                   if (!smgrexists(rels[i], j))
+                           continue;
+
+                   /* Get the number of blocks for a relation's fork */
+                   blocks[i][numForks] = smgrnblocks(rels[i], j,
NULL);

If we see a fork which its size is not cached we must give up this
optimization for all target relations.

I did not use the "cached" flag in DropRelFileNodesAllBuffers and
use InRecovery when deciding for optimization because of the following

reasons:

XLogReadBufferExtended() calls smgrnblocks() to apply changes to
relation page contents. So in DropRelFileNodeBuffers(),
XLogReadBufferExtended() is called during VACUUM replay because

VACUUM changes the page content.

OTOH, TRUNCATE doesn't change the relation content, it just
truncates relation pages without changing the page contents. So
XLogReadBufferExtended() is not called, and the "cached" flag will
always return false. I tested with "cached" flags before, and this

A bit different from the point, but if some tuples have been inserted
to the truncated table, XLogReadBufferExtended() is called for the
table and the length is cached.

always return false, at least in DropRelFileNodesAllBuffers. Due to
this, we cannot use the cached flag in DropRelFileNodesAllBuffers().
However, I think we can still rely on smgrnblocks to get the file
size as long as we're InRecovery. That cached nblocks is still guaranteed

to be the maximum in the shared buffer.

Thoughts?

That means that we always think as if smgrnblocks returns "cached" (or
"safe") value during recovery, which is out of our current consensus.
If we go on that side, we don't need to consult the "cached" returned
from smgrnblocks at all and it's enough to see only InRecovery.

I got confused..

We are relying on the "fact" that the first lseek() call of a
(startup) process tells the truth. We added an assertion so that we
make sure that the cached value won't be cleared during recovery. A
possible remaining danger would be closing of an smgr object of a live
relation just after a file extension failure. I think we are thinking
that that doesn't happen during recovery. Although it seems to me
true, I'm not confident.

Yeah, I also think it might not be worth depending upon whether smgr close
has been done before or not. I feel the current idea of using 'cached'
parameter is relatively solid and we should rely on that.
Also, which means that in DropRelFileNodesAllBuffers() we should rely on
the same, I think doing things differently in this regard will lead to confusion. I
agree in some cases we might not get benefits but it is more important to be
correct and keep the code consistent to avoid introducing bugs now or in the
future.

Hi,
I have reported before that it is not always the case that the "cached" flag of
srnblocks() return true. So when I checked the truncate test case used in my
patch, it does not enter the optimization path despite doing INSERT before
truncation of table.
The reason for that is because in TRUNCATE, a new RelFileNode is assigned
to the relation when creating a new file. In recovery, XLogReadBufferExtended()
always opens the RelFileNode and calls smgrnblocks() for that RelFileNode for the
first time. And for recovery processing, different RelFileNodes are used for the
INSERTs to the table and TRUNCATE to the same table.

As we cannot use "cached" flag for both DropRelFileNodeBuffers() and
DropRelFileNodesAllBuffers() based from above.
I am thinking that if we want consistency, correctness, and to still make use of
the optimization, we can completely drop the "cached" flag parameter in smgrnblocks,
and use InRecovery.
Tsunakawa-san mentioned in [1]/messages/by-id/TYAPR01MB2990B42570A5FAC349EE983AFEF40@TYAPR01MB2990.jpnprd01.prod.outlook.com that it is safe because smgrclose is not called
by the startup process in recovery. Shared-inval messages are not sent to startup
process.

Otherwise, we use the current patch form as it is: using "cached" in
DropRelFileNodeBuffers() and InRecovery for DropRelFileNodesAllBuffers().
However, that does not seem to be what is wanted in this thread.

Thoughts?

Regards,
Kirk Jamison

[1]: /messages/by-id/TYAPR01MB2990B42570A5FAC349EE983AFEF40@TYAPR01MB2990.jpnprd01.prod.outlook.com

#186Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#185)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Mon, Dec 7, 2020 at 12:32 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Friday, December 4, 2020 8:27 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:

On Fri, Nov 27, 2020 at 11:36 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Fri, 27 Nov 2020 02:19:57 +0000, "k.jamison@fujitsu.com"
<k.jamison@fujitsu.com> wrote in

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com> Hello, Kirk.
Thank you for the new version.

Hi, Horiguchi-san. Thank you for your very helpful feedback.
I'm updating the patches addressing those.

+                   if (!smgrexists(rels[i], j))
+                           continue;
+
+                   /* Get the number of blocks for a relation's fork */
+                   blocks[i][numForks] = smgrnblocks(rels[i], j,
NULL);

If we see a fork which its size is not cached we must give up this
optimization for all target relations.

I did not use the "cached" flag in DropRelFileNodesAllBuffers and
use InRecovery when deciding for optimization because of the following

reasons:

XLogReadBufferExtended() calls smgrnblocks() to apply changes to
relation page contents. So in DropRelFileNodeBuffers(),
XLogReadBufferExtended() is called during VACUUM replay because

VACUUM changes the page content.

OTOH, TRUNCATE doesn't change the relation content, it just
truncates relation pages without changing the page contents. So
XLogReadBufferExtended() is not called, and the "cached" flag will
always return false. I tested with "cached" flags before, and this

A bit different from the point, but if some tuples have been inserted
to the truncated table, XLogReadBufferExtended() is called for the
table and the length is cached.

always return false, at least in DropRelFileNodesAllBuffers. Due to
this, we cannot use the cached flag in DropRelFileNodesAllBuffers().
However, I think we can still rely on smgrnblocks to get the file
size as long as we're InRecovery. That cached nblocks is still guaranteed

to be the maximum in the shared buffer.

Thoughts?

That means that we always think as if smgrnblocks returns "cached" (or
"safe") value during recovery, which is out of our current consensus.
If we go on that side, we don't need to consult the "cached" returned
from smgrnblocks at all and it's enough to see only InRecovery.

I got confused..

We are relying on the "fact" that the first lseek() call of a
(startup) process tells the truth. We added an assertion so that we
make sure that the cached value won't be cleared during recovery. A
possible remaining danger would be closing of an smgr object of a live
relation just after a file extension failure. I think we are thinking
that that doesn't happen during recovery. Although it seems to me
true, I'm not confident.

Yeah, I also think it might not be worth depending upon whether smgr close
has been done before or not. I feel the current idea of using 'cached'
parameter is relatively solid and we should rely on that.
Also, which means that in DropRelFileNodesAllBuffers() we should rely on
the same, I think doing things differently in this regard will lead to confusion. I
agree in some cases we might not get benefits but it is more important to be
correct and keep the code consistent to avoid introducing bugs now or in the
future.

Hi,
I have reported before that it is not always the case that the "cached" flag of
srnblocks() return true. So when I checked the truncate test case used in my
patch, it does not enter the optimization path despite doing INSERT before
truncation of table.
The reason for that is because in TRUNCATE, a new RelFileNode is assigned
to the relation when creating a new file. In recovery, XLogReadBufferExtended()
always opens the RelFileNode and calls smgrnblocks() for that RelFileNode for the
first time. And for recovery processing, different RelFileNodes are used for the
INSERTs to the table and TRUNCATE to the same table.

Hmm, how is it possible if Insert is done before Truncate? The insert
should happen in old RelFileNode only. I have verified by adding a
break-in (while (1), so that it stops there) heap_xlog_insert and
DropRelFileNodesAllBuffers(), and both get the same (old) RelFileNode.
How have you verified what you are saying?

--
With Regards,
Amit Kapila.

#187Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#186)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Mon, 7 Dec 2020 17:18:31 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Mon, Dec 7, 2020 at 12:32 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Friday, December 4, 2020 8:27 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
Hi,
I have reported before that it is not always the case that the "cached" flag of
srnblocks() return true. So when I checked the truncate test case used in my
patch, it does not enter the optimization path despite doing INSERT before
truncation of table.
The reason for that is because in TRUNCATE, a new RelFileNode is assigned
to the relation when creating a new file. In recovery, XLogReadBufferExtended()
always opens the RelFileNode and calls smgrnblocks() for that RelFileNode for the
first time. And for recovery processing, different RelFileNodes are used for the
INSERTs to the table and TRUNCATE to the same table.

Hmm, how is it possible if Insert is done before Truncate? The insert
should happen in old RelFileNode only. I have verified by adding a
break-in (while (1), so that it stops there) heap_xlog_insert and
DropRelFileNodesAllBuffers(), and both get the same (old) RelFileNode.
How have you verified what you are saying?

You might be thinking of in-transaction sequence of
Inert-truncate. What *I* mention before is truncation of a relation
that smgrnblocks() has already been called for. The most common way
to make it happen was INSERTs *before* the truncating transaction
starts. It may be a SELECT on a hot-standby. Sorry for the confusing
expression.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#188Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Kyotaro Horiguchi (#187)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Tue, 08 Dec 2020 09:45:53 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

At Mon, 7 Dec 2020 17:18:31 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

Hmm, how is it possible if Insert is done before Truncate? The insert
should happen in old RelFileNode only. I have verified by adding a
break-in (while (1), so that it stops there) heap_xlog_insert and
DropRelFileNodesAllBuffers(), and both get the same (old) RelFileNode.
How have you verified what you are saying?

You might be thinking of in-transaction sequence of
Inert-truncate. What *I* mention before is truncation of a relation
that smgrnblocks() has already been called for. The most common way
to make it happen was INSERTs *before* the truncating transaction
starts. It may be a SELECT on a hot-standby. Sorry for the confusing
expression.

And ,to make sure, it is a bit off from the point of the discussion as
I noted. I just meant that the proposition that "smgrnblokcs() always
returns false for "cached" when it is called in
DropRelFileNodesAllBuffers()" doesn't always holds.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#189Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#188)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Dec 8, 2020 at 6:23 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Tue, 08 Dec 2020 09:45:53 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

At Mon, 7 Dec 2020 17:18:31 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

Hmm, how is it possible if Insert is done before Truncate? The insert
should happen in old RelFileNode only. I have verified by adding a
break-in (while (1), so that it stops there) heap_xlog_insert and
DropRelFileNodesAllBuffers(), and both get the same (old) RelFileNode.
How have you verified what you are saying?

You might be thinking of in-transaction sequence of
Inert-truncate. What *I* mention before is truncation of a relation
that smgrnblocks() has already been called for. The most common way
to make it happen was INSERTs *before* the truncating transaction
starts.

What I have tried is Insert and Truncate in separate transactions like below:
postgres=# insert into mytbl values(1);
INSERT 0 1
postgres=# truncate mytbl;
TRUNCATE TABLE

After above, manually killed the server, and then during recovery, we
have called heap_xlog_insert() and DropRelFileNodesAllBuffers() and at
both places, RelFileNode is the same and I don't see any reason for it
to be different.

It may be a SELECT on a hot-standby. Sorry for the confusing
expression.

And ,to make sure, it is a bit off from the point of the discussion as
I noted. I just meant that the proposition that "smgrnblokcs() always
returns false for "cached" when it is called in
DropRelFileNodesAllBuffers()" doesn't always holds.

Right, I feel in some cases the 'cached' won't be true like if we
would have done Checkpoint after Insert in the above case (say when
the only WAL to replay during recovery is of Truncate) but I think
that should be fine. What do you think?

--
With Regards,
Amit Kapila.

#190Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Kyotaro Horiguchi (#187)
Re: [Patch] Optimize dropping of relation buffers using dlist

I'm out of it more than usual..

At Tue, 08 Dec 2020 09:45:53 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

At Mon, 7 Dec 2020 17:18:31 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Mon, Dec 7, 2020 at 12:32 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Friday, December 4, 2020 8:27 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
Hi,
I have reported before that it is not always the case that the "cached" flag of
srnblocks() return true. So when I checked the truncate test case used in my
patch, it does not enter the optimization path despite doing INSERT before
truncation of table.
The reason for that is because in TRUNCATE, a new RelFileNode is assigned
to the relation when creating a new file. In recovery, XLogReadBufferExtended()
always opens the RelFileNode and calls smgrnblocks() for that RelFileNode for the
first time. And for recovery processing, different RelFileNodes are used for the
INSERTs to the table and TRUNCATE to the same table.

Hmm, how is it possible if Insert is done before Truncate? The insert
should happen in old RelFileNode only. I have verified by adding a
break-in (while (1), so that it stops there) heap_xlog_insert and
DropRelFileNodesAllBuffers(), and both get the same (old) RelFileNode.
How have you verified what you are saying?

It's irrelvant that the insert happens on the old relfilenode. We drop
buffers for the old relfilenode on truncation anyway.

What I did is:

a: Create a physical replication pair.
b: On the master, create a table. (without explicitly starting a tx)
c: On the master, insert a tuple into the table.
d: On the master truncate the table.

On the standby, smgrnblocks is called for the old relfilenode of the
table at c, then the same function is called for the same relfilenode
at d and the function takes the cached path.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#191Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#190)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Dec 8, 2020 at 7:24 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

I'm out of it more than usual..

At Tue, 08 Dec 2020 09:45:53 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

At Mon, 7 Dec 2020 17:18:31 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Mon, Dec 7, 2020 at 12:32 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Friday, December 4, 2020 8:27 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
Hi,
I have reported before that it is not always the case that the "cached" flag of
srnblocks() return true. So when I checked the truncate test case used in my
patch, it does not enter the optimization path despite doing INSERT before
truncation of table.
The reason for that is because in TRUNCATE, a new RelFileNode is assigned
to the relation when creating a new file. In recovery, XLogReadBufferExtended()
always opens the RelFileNode and calls smgrnblocks() for that RelFileNode for the
first time. And for recovery processing, different RelFileNodes are used for the
INSERTs to the table and TRUNCATE to the same table.

Hmm, how is it possible if Insert is done before Truncate? The insert
should happen in old RelFileNode only. I have verified by adding a
break-in (while (1), so that it stops there) heap_xlog_insert and
DropRelFileNodesAllBuffers(), and both get the same (old) RelFileNode.
How have you verified what you are saying?

It's irrelvant that the insert happens on the old relfilenode.

I think it is relevant because it will allow the 'blocks' value to be cached.

We drop
buffers for the old relfilenode on truncation anyway.

What I did is:

a: Create a physical replication pair.
b: On the master, create a table. (without explicitly starting a tx)
c: On the master, insert a tuple into the table.
d: On the master truncate the table.

On the standby, smgrnblocks is called for the old relfilenode of the
table at c, then the same function is called for the same relfilenode
at d and the function takes the cached path.

This is on the lines I have tried for recovery. So, it seems we are in
agreement that we can use the 'cached' flag in
DropRelFileNodesAllBuffers and it will take the optimized path in many
such cases, right?

--
With Regards,
Amit Kapila.

#192Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#191)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Tue, 8 Dec 2020 08:08:25 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Tue, Dec 8, 2020 at 7:24 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

We drop
buffers for the old relfilenode on truncation anyway.

What I did is:

a: Create a physical replication pair.
b: On the master, create a table. (without explicitly starting a tx)
c: On the master, insert a tuple into the table.
d: On the master truncate the table.

On the standby, smgrnblocks is called for the old relfilenode of the
table at c, then the same function is called for the same relfilenode
at d and the function takes the cached path.

This is on the lines I have tried for recovery. So, it seems we are in
agreement that we can use the 'cached' flag in
DropRelFileNodesAllBuffers and it will take the optimized path in many
such cases, right?

Mmm. There seems to be a misunderstanding.. What I opposed to is
referring only to InRecovery and ignoring the value of "cached".

The remaining issue is we don't get to the optimized path when a
standby makes the first call to smgrnblocks() when truncating a
relation. Still we can get to the optimized path as far as any
update(+insert) or select is performed earlier on the relation so I
think it doesn't matter so match.

But I'm not sure what others think.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#193Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#192)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Dec 8, 2020 at 10:41 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Tue, 8 Dec 2020 08:08:25 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Tue, Dec 8, 2020 at 7:24 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

We drop
buffers for the old relfilenode on truncation anyway.

What I did is:

a: Create a physical replication pair.
b: On the master, create a table. (without explicitly starting a tx)
c: On the master, insert a tuple into the table.
d: On the master truncate the table.

On the standby, smgrnblocks is called for the old relfilenode of the
table at c, then the same function is called for the same relfilenode
at d and the function takes the cached path.

This is on the lines I have tried for recovery. So, it seems we are in
agreement that we can use the 'cached' flag in
DropRelFileNodesAllBuffers and it will take the optimized path in many
such cases, right?

Mmm. There seems to be a misunderstanding.. What I opposed to is
referring only to InRecovery and ignoring the value of "cached".

Okay, I think it was Kirk-San who proposed to use InRecovery and
ignoring the value of "cached" based on the theory that even if Insert
(or other DMLs) are done before Truncate, it won't use an optimized
path and I don't agree with the same. So, I did a small test to check
the same and found that it should use the optimized path and the same
is true for the experiment done by you. I am not sure why Kirk-San is
seeing something different?

The remaining issue is we don't get to the optimized path when a
standby makes the first call to smgrnblocks() when truncating a
relation. Still we can get to the optimized path as far as any
update(+insert) or select is performed earlier on the relation so I
think it doesn't matter so match.

+1.

With Regards,
Amit Kapila.

#194k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#193)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Tuesday, December 8, 2020 2:35 PM, Amit Kapila wrote:

On Tue, Dec 8, 2020 at 10:41 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Tue, 8 Dec 2020 08:08:25 +0530, Amit Kapila
<amit.kapila16@gmail.com> wrote in

On Tue, Dec 8, 2020 at 7:24 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

We drop
buffers for the old relfilenode on truncation anyway.

What I did is:

a: Create a physical replication pair.
b: On the master, create a table. (without explicitly starting a
tx)
c: On the master, insert a tuple into the table.
d: On the master truncate the table.

On the standby, smgrnblocks is called for the old relfilenode of
the table at c, then the same function is called for the same
relfilenode at d and the function takes the cached path.

This is on the lines I have tried for recovery. So, it seems we are
in agreement that we can use the 'cached' flag in
DropRelFileNodesAllBuffers and it will take the optimized path in
many such cases, right?

Mmm. There seems to be a misunderstanding.. What I opposed to is
referring only to InRecovery and ignoring the value of "cached".

Okay, I think it was Kirk-San who proposed to use InRecovery and ignoring
the value of "cached" based on the theory that even if Insert (or other DMLs)
are done before Truncate, it won't use an optimized path and I don't agree
with the same. So, I did a small test to check the same and found that it
should use the optimized path and the same is true for the experiment done
by you. I am not sure why Kirk-San is seeing something different?

The remaining issue is we don't get to the optimized path when a
standby makes the first call to smgrnblocks() when truncating a
relation. Still we can get to the optimized path as far as any
update(+insert) or select is performed earlier on the relation so I
think it doesn't matter so match.

+1.

My question/proposal before was to either use InRecovery,
or completely drop the smgrnblocks' "cached" flag.
But that is coming from the results of my investigation below when
I used "cached" in DropRelFileNodesAllBuffers().
The optimization path was skipped because one of the
Rels' "cached" value was "false".

Test Case. (shared_buffer = 1GB)
0. Set physical replication to both master and standby.
1. Create 1 table.
2. Insert Data (1MB) to TABLE.
16385 is the relnode for insert (both Master and Standby).

3. Pause WAL on Standby.
4. TRUNCATE table on Primary.
nrels = 3. relNodes 16389, 16388, 16385.

5. Stop Primary.

6. Promote standby and resume WAL recovery. nrels = 3
1st rel's check for optimization: "cached" is TRUE. relNode = 16389.
2nd rel's check for optimization. "cached" was returned FALSE by
smgrnblocks). relNode = 16388.
Since one of the rels' cached is "FALSE", the optimization check for
3rd relation and the whole optimization itself is skipped.
Go to full-scan path in DropRelFileNodesAllBuffers().
Then smgrclose for relNodes 16389, 16388, 16385.

Because one of the rel's cached value was false, it forced the
full-scan path for TRUNCATE.
Is there a possible workaround for this?

Regards,
Kirk Jamison

#195tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#194)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Because one of the rel's cached value was false, it forced the
full-scan path for TRUNCATE.
Is there a possible workaround for this?

Hmm, the other two relfilenodes are for the TOAST table and index of the target table. I think the INSERT didn't access those TOAST relfilenodes because the inserted data was stored in the main storage. But TRUNCATE always truncates all the three relfilenodes. So, the standby had not opened the relfilenode for the TOAST stuff or cached its size when replaying the TRUNCATE.

I'm afraid this is more common than we can ignore and accept the slow traditional path, but I don't think of a good idea to use the cached flag.

Regards
Takayuki Tsunakawa

#196Amit Kapila
amit.kapila16@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#195)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Dec 8, 2020 at 12:13 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Because one of the rel's cached value was false, it forced the
full-scan path for TRUNCATE.
Is there a possible workaround for this?

Hmm, the other two relfilenodes are for the TOAST table and index of the target table. I think the INSERT didn't access those TOAST relfilenodes because the inserted data was stored in the main storage. But TRUNCATE always truncates all the three relfilenodes. So, the standby had not opened the relfilenode for the TOAST stuff or cached its size when replaying the TRUNCATE.

I'm afraid this is more common than we can ignore and accept the slow traditional path, but I don't think of a good idea to use the cached flag.

I also can't think of a way to use an optimized path for such cases
but I don't agree with your comment on if it is common enough that we
leave this optimization entirely for the truncate path.

--
With Regards,
Amit Kapila.

#197Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#196)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Tue, 8 Dec 2020 16:28:41 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Tue, Dec 8, 2020 at 12:13 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Because one of the rel's cached value was false, it forced the
full-scan path for TRUNCATE.
Is there a possible workaround for this?

Hmm, the other two relfilenodes are for the TOAST table and index of the target table. I think the INSERT didn't access those TOAST relfilenodes because the inserted data was stored in the main storage. But TRUNCATE always truncates all the three relfilenodes. So, the standby had not opened the relfilenode for the TOAST stuff or cached its size when replaying the TRUNCATE.

I'm afraid this is more common than we can ignore and accept the slow traditional path, but I don't think of a good idea to use the cached flag.

I also can't think of a way to use an optimized path for such cases
but I don't agree with your comment on if it is common enough that we
leave this optimization entirely for the truncate path.

Mmm. At least btree doesn't need to call smgrnblocks except at
expansion, so we cannot get to the optimized path in major cases of
truncation involving btree (and/or maybe other indexes). TOAST
relations are not accessed until we insert/update/retrive the values
in it.

An ugly way to cope with it would be to let other smgr functions
manage the cached value, for example, by calling smgrnblocks while
InRecovery. Or letting smgr remember the maximum block number ever
accessed. But we cannot fully rely on that since smgr can be closed
midst of a session and smgr doesn't offer such persistence. In the
first place smgr doesn't seem to be the place to store such persistent
information.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#198tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Kyotaro Horiguchi (#197)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

At Tue, 8 Dec 2020 16:28:41 +0530, Amit Kapila <amit.kapila16@gmail.com>
wrote in
I also can't think of a way to use an optimized path for such cases

but I don't agree with your comment on if it is common enough that we
leave this optimization entirely for the truncate path.

An ugly way to cope with it would be to let other smgr functions
manage the cached value, for example, by calling smgrnblocks while
InRecovery. Or letting smgr remember the maximum block number ever
accessed. But we cannot fully rely on that since smgr can be closed
midst of a session and smgr doesn't offer such persistence. In the
first place smgr doesn't seem to be the place to store such persistent
information.

Yeah, considering the future evolution of this patch to operations during normal running, I don't think that would be a good fit, either.

Then, the as we're currently targeting just recovery, the options we can take are below. Which would vote for? My choice would be (3) > (2) > (1).

(1)
Use the cached flag in both VACUUM (0003) and TRUNCATE (0004).
This brings the most safety and code consistency.
But this would not benefit from optimization for TRUNCATE in unexpectedly many cases -- when TOAST storage exists but it's not written, or FSM/VM is not updated after checkpoint.

(2)
Use the cached flag in VACUUM (0003), but use InRecovery instead of the cached flag in TRUNCATE (0004).
This benefits from the optimization in all cases.
But this lacks code consistency.
You may be afraid of safety if the startup process smgrclose()s the relation after the shared buffer flushing hits disk full. However, startup process doesn't smgrclose(), so it should be safe. Just in case the startup process smgrclose()s, the worst consequence is PANIC shutdown after repeated failure of checkpoints due to lingering orphaned dirty shared buffers. Accept it as Thomas-san's devil's suggestion.

(3)
Do not use the cached flag in either VACUUM (0003) or TRUNCATE (0004).
This benefits from the optimization in all cases.
The code is consistent and smaller.
As for the safety, this is the same as (2), but it applies to VACUUM as well.

Regards
Takayuki Tsunakawa

#199k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#198)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wednesday, December 9, 2020 10:58 AM, Tsunakawa, Takayuki wrote:

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

At Tue, 8 Dec 2020 16:28:41 +0530, Amit Kapila
<amit.kapila16@gmail.com> wrote in
I also can't think of a way to use an optimized path for such cases

but I don't agree with your comment on if it is common enough that
we leave this optimization entirely for the truncate path.

An ugly way to cope with it would be to let other smgr functions
manage the cached value, for example, by calling smgrnblocks while
InRecovery. Or letting smgr remember the maximum block number ever
accessed. But we cannot fully rely on that since smgr can be closed
midst of a session and smgr doesn't offer such persistence. In the
first place smgr doesn't seem to be the place to store such persistent
information.

Yeah, considering the future evolution of this patch to operations during
normal running, I don't think that would be a good fit, either.

Then, the as we're currently targeting just recovery, the options we can take
are below. Which would vote for? My choice would be (3) > (2) > (1).

(1)
Use the cached flag in both VACUUM (0003) and TRUNCATE (0004).
This brings the most safety and code consistency.
But this would not benefit from optimization for TRUNCATE in unexpectedly
many cases -- when TOAST storage exists but it's not written, or FSM/VM is
not updated after checkpoint.

(2)
Use the cached flag in VACUUM (0003), but use InRecovery instead of the
cached flag in TRUNCATE (0004).
This benefits from the optimization in all cases.
But this lacks code consistency.
You may be afraid of safety if the startup process smgrclose()s the relation
after the shared buffer flushing hits disk full. However, startup process
doesn't smgrclose(), so it should be safe. Just in case the startup process
smgrclose()s, the worst consequence is PANIC shutdown after repeated
failure of checkpoints due to lingering orphaned dirty shared buffers. Accept
it as Thomas-san's devil's suggestion.

(3)
Do not use the cached flag in either VACUUM (0003) or TRUNCATE (0004).
This benefits from the optimization in all cases.
The code is consistent and smaller.
As for the safety, this is the same as (2), but it applies to VACUUM as well.

If we want code consistency, then we'd fall in either 1 or 3.
And if we want to take the benefits of optimization for both DropRelFileNodeBuffers
and DropRelFileNodesAllBuffers, then I'd choose 3.
However, if the reviewers and committer want to make use of the "cached" flag,
then we can live with "cached" value in place there even if it's not common to
get the optimization for TRUNCATE path. So only VACUUM would take the most
benefit.
My vote is also (3), then (2), and (1).

Regards,
Kirk Jamison

#200Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#197)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Dec 9, 2020 at 6:32 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Tue, 8 Dec 2020 16:28:41 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Tue, Dec 8, 2020 at 12:13 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Because one of the rel's cached value was false, it forced the
full-scan path for TRUNCATE.
Is there a possible workaround for this?

Hmm, the other two relfilenodes are for the TOAST table and index of the target table. I think the INSERT didn't access those TOAST relfilenodes because the inserted data was stored in the main storage. But TRUNCATE always truncates all the three relfilenodes. So, the standby had not opened the relfilenode for the TOAST stuff or cached its size when replaying the TRUNCATE.

I'm afraid this is more common than we can ignore and accept the slow traditional path, but I don't think of a good idea to use the cached flag.

I also can't think of a way to use an optimized path for such cases
but I don't agree with your comment on if it is common enough that we
leave this optimization entirely for the truncate path.

Mmm. At least btree doesn't need to call smgrnblocks except at
expansion, so we cannot get to the optimized path in major cases of
truncation involving btree (and/or maybe other indexes).

AFAICS, btree insert should call smgrnblocks via
btree_xlog_insert->XLogReadBufferForRedo->XLogReadBufferForRedoExtended->XLogReadBufferExtended->smgrnblocks.
Similarly delete should also call smgrnblocks. Can you be bit more
specific related to the btree case you have in mind?

--
With Regards,
Amit Kapila.

#201Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#200)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Wed, 9 Dec 2020 16:27:30 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Wed, Dec 9, 2020 at 6:32 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Tue, 8 Dec 2020 16:28:41 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Tue, Dec 8, 2020 at 12:13 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Because one of the rel's cached value was false, it forced the
full-scan path for TRUNCATE.
Is there a possible workaround for this?

Hmm, the other two relfilenodes are for the TOAST table and index of the target table. I think the INSERT didn't access those TOAST relfilenodes because the inserted data was stored in the main storage. But TRUNCATE always truncates all the three relfilenodes. So, the standby had not opened the relfilenode for the TOAST stuff or cached its size when replaying the TRUNCATE.

I'm afraid this is more common than we can ignore and accept the slow traditional path, but I don't think of a good idea to use the cached flag.

I also can't think of a way to use an optimized path for such cases
but I don't agree with your comment on if it is common enough that we
leave this optimization entirely for the truncate path.

Mmm. At least btree doesn't need to call smgrnblocks except at
expansion, so we cannot get to the optimized path in major cases of
truncation involving btree (and/or maybe other indexes).

AFAICS, btree insert should call smgrnblocks via
btree_xlog_insert->XLogReadBufferForRedo->XLogReadBufferForRedoExtended->XLogReadBufferExtended->smgrnblocks.
Similarly delete should also call smgrnblocks. Can you be bit more
specific related to the btree case you have in mind?

Oh, sorry. I wrongly looked to non-recovery path. smgrnblocks is
called during buffer loading while recovery. So, smgrnblock is called
for indexes if any update happens on the heap relation.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#202tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Kyotaro Horiguchi (#201)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

Oh, sorry. I wrongly looked to non-recovery path. smgrnblocks is
called during buffer loading while recovery. So, smgrnblock is called
for indexes if any update happens on the heap relation.

I misunderstood that you said there's no problem with the TOAST index because TRUNCATE creates the meta page, resulting in the caching of the page and size of the relation. Anyway, I'm relieved the concern disappeared.

Then, I'd like to hear your vote in my previous mail...

Regards
Takayuki Tsunakawa

#203Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#201)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Dec 10, 2020 at 7:11 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Wed, 9 Dec 2020 16:27:30 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Wed, Dec 9, 2020 at 6:32 AM Kyotaro Horiguchi

Mmm. At least btree doesn't need to call smgrnblocks except at
expansion, so we cannot get to the optimized path in major cases of
truncation involving btree (and/or maybe other indexes).

AFAICS, btree insert should call smgrnblocks via
btree_xlog_insert->XLogReadBufferForRedo->XLogReadBufferForRedoExtended->XLogReadBufferExtended->smgrnblocks.
Similarly delete should also call smgrnblocks. Can you be bit more
specific related to the btree case you have in mind?

Oh, sorry. I wrongly looked to non-recovery path. smgrnblocks is
called during buffer loading while recovery. So, smgrnblock is called
for indexes if any update happens on the heap relation.

Okay, so this means that we can get the benefit of optimization in
many cases in the Truncate code path as well even if we use 'cached'
flag? If so, then I would prefer to keep the code consistent for both
vacuum and truncate recovery code path.

--
With Regards,
Amit Kapila.

#204k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#203)
4 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, December 10, 2020 12:27 PM, Amit Kapila wrote:

On Thu, Dec 10, 2020 at 7:11 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Wed, 9 Dec 2020 16:27:30 +0530, Amit Kapila
<amit.kapila16@gmail.com> wrote in

On Wed, Dec 9, 2020 at 6:32 AM Kyotaro Horiguchi

Mmm. At least btree doesn't need to call smgrnblocks except at
expansion, so we cannot get to the optimized path in major cases
of truncation involving btree (and/or maybe other indexes).

AFAICS, btree insert should call smgrnblocks via

btree_xlog_insert->XLogReadBufferForRedo->XLogReadBufferForRedoExte
nded->XLogReadBufferExtended->smgrnblocks.

Similarly delete should also call smgrnblocks. Can you be bit more
specific related to the btree case you have in mind?

Oh, sorry. I wrongly looked to non-recovery path. smgrnblocks is
called during buffer loading while recovery. So, smgrnblock is called
for indexes if any update happens on the heap relation.

Okay, so this means that we can get the benefit of optimization in many cases
in the Truncate code path as well even if we use 'cached'
flag? If so, then I would prefer to keep the code consistent for both vacuum
and truncate recovery code path.

Yes, I have tested that optimization works for index relations.

I have attached the V34, following the conditions that we use "cached" flag
for both DropRelFileNodesBuffers() and DropRelFileNodesBuffers() for
consistency.
I added comment in 0004 the limitation of optimization when there are TOAST
relations that use NON-PLAIN strategy. i.e. The optimization works if the data
types used are integers, OID, bytea, etc. But for TOAST-able data types like text,
the optimization will be skipped and force a full scan during recovery.

Regards,
Kirk Jamison

Attachments:

v34-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v34-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From 618eb0cf48f0a6146330ba643007134e8258ecd1 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v34 1/4] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v34-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v34-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From 8ffe3aacc88baafb5376804da2558d1017c99169 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v34 2/4] Add bool param in smgrnblocks() for cached blocks.

The flag ensures that we return a reliable value from smgrnblocks.
---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 21 ++++++++++++++++++---
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 34 insertions(+), 18 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 32a3099..afc640d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index d538f257..3874ff3 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -721,7 +721,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ad0d1a9..1680bf4 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -740,7 +740,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2857,7 +2857,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..e9dffd2 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -535,7 +535,6 @@ smgrwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 										buffer, skipFsync);
 }
 
-
 /*
  *	smgrwriteback() -- Trigger kernel writeback for the supplied range of
  *					   blocks.
@@ -551,18 +550,34 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 /*
  *	smgrnblocks() -- Calculate the number of blocks in the
  *					 supplied relation.
+ *
+ *		The "cached" flag ensures that no buffers exist for blocks after the
+ *		cached value is known to the process during recovery.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *cached)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  In recovery, the cached
+	 * value returned by the first lseek could be smaller than the actual number
+	 * of existing buffers of the file, which is caused by buggy Linux kernels
+	 * that might not have accounted for the recent write.  However, we can still
+	 * rely on the cached value even if we get a bogus value from first lseek
+	 * since it is impossible to have buffer for blocks after that file size.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (cached != NULL)
+			*cached = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
+
+	if (cached != NULL)
+		*cached = false;
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..cd99f1b 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *cached);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v34-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v34-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From d3137b924d2b9d8685923a0b1285ff279b744780 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Tue, 10 Nov 2020 06:27:11 +0000
Subject: [PATCH v34 3/4] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning. This improves the
performance especially when VACUUM or autovacuum truncated off any of
the empty pages at the end of relation.

While recovery, when WAL files of XLOG_SMGR_TRUNCATE from vacuum or
autovacuum are replayed, we must not leave a buffer for the relations
to be dropped.  So we invalidate buffer blocks by locating using
BufTableLookup() when it is certain that we know up to what page of
every fork we possiblly have a buffer.  We can know that by the
"cached" flag returned by smgrnblocks(), which currently gets true
only while recovery.  It's safe to use the cached nblocks because it
is guaranteed to be the maximum page we have in shared buffer during
recovery, and that guarantee is held by not asking fseek once we have
cached the value.  Also, the cached nblocks will not be invalidated
by file extension during recovery. See smgrnblocks() and smgrextend()
for more details.

If we are not in recovery or when the nblocks is not cached, then we
scan the whole buffer pool to search and drop the buffers of relation,
which is slower when a small part of buffers are to be dropped.
---
 src/backend/storage/buffer/bufmgr.c | 115 +++++++++++++++++++++++++++++++++---
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 110 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 1680bf4..c3ee6c6 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 256)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,10 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FindAndDropRelFileNodeBuffers(RelFileNode rnode,
+										  ForkNumber forkNum,
+										  BlockNumber nForkBlock,
+										  BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2967,18 +2973,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		We must not leave a buffer for the relations to be dropped.  We
+ *		invalidate buffer blocks by locating using BufTableLookup() when it
+ *		is certain that we know up to what page of every fork we possiblly
+ *		have a buffer.  We can know that by the "cached" flag returned by
+ *		smgrnblocks(), which currently gets true only while recovery.  See
+ *		smgrnblocks() and smgrextend().  Otherwise we scan the whole buffer
+ *		pool to find buffers for the relation, which is slower when a small
+ *		part of buffers are to be dropped.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		cached = false;
+	BlockNumber	nForkBlock[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2992,6 +3008,38 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of to-be-invalidated blocks of a relation as
+	 * well as the total blocks for a given fork.  Give up the optimization
+	 * if the block is not cached.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlock[i] = smgrnblocks(smgr_reln, forkNum[i], &cached);
+
+		if (!cached)
+			break;
+
+		/* Get the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlock[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffers in the hashtable and drop them if the block size
+	 * is already cached and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			FindAndDropRelFileNodeBuffers(rnode.node, forkNum[j],
+										  nForkBlock[j], firstDelBlock[j]);
+		}
+		return;
+	}
+
 	for (i = 0; i < NBuffers; i++)
 	{
 		BufferDesc *bufHdr = GetBufferDescriptor(i);
@@ -3135,6 +3183,60 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 }
 
 /* ---------------------------------------------------------------------
+ *		FindAndDropRelFileNodeBuffers
+ *
+ *		This function finds and removes from the buffer pool all the pages
+ *		of the specified relation fork that has block number >= firstDelBlock.
+ *		(In particular, with firstDelBlock = 0, all pages are removed.)
+ *		This is only called in recovery when the block count of a fork is
+ *		cached or when the total number of to-be-invalidated blocks for
+ *		involved relation/s does not exceed the threshold for buffer full scan.
+ * --------------------------------------------------------------------
+ */
+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber forkNum,
+							  BlockNumber nForkBlock, BlockNumber firstDelBlock)
+{
+	BlockNumber		curBlock;
+
+	for (curBlock = firstDelBlock; curBlock < nForkBlock; curBlock++)
+	{
+		uint32		bufHash;		/* hash value for tag */
+		BufferTag	bufTag;			/* identity of requested block */
+		LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+		int		buf_id;
+		BufferDesc	*bufHdr;
+		uint32		buf_state;
+
+		/* create a tag so we can lookup the buffer */
+		INIT_BUFFERTAG(bufTag, rnode, forkNum, curBlock);
+
+		/* determine its hash code and partition lock ID */
+		bufHash = BufTableHashCode(&bufTag);
+		bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+		/* Check that it is in the buffer pool. If not, do nothing. */
+		LWLockAcquire(bufPartitionLock, LW_SHARED);
+		buf_id = BufTableLookup(&bufTag, bufHash);
+		LWLockRelease(bufPartitionLock);
+
+		if (buf_id < 0)
+			continue;
+
+		bufHdr = GetBufferDescriptor(buf_id);
+
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+/* ---------------------------------------------------------------------
  *		DropDatabaseBuffers
  *
  *		This function removes all the buffers in the buffer cache for a
@@ -3246,8 +3348,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index e9dffd2..9d3a67c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -605,7 +605,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v34-0004-Optimize-DropRelFileNodesAllBuffers-in-recovery.patchapplication/octet-stream; name=v34-0004-Optimize-DropRelFileNodesAllBuffers-in-recovery.patchDownload
From f3183b374f5e6b0e5732c313585d8507e1143082 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 2 Dec 2020 12:48:04 +0000
Subject: [PATCH v34 4/4] Optimize DropRelFileNodesAllBuffers() in recovery.

DropRelFileNodesAllBuffers() is optimized to skip the time-consuming
scan of the whole buffer pool during recovery when the relation is
small enough, or when the number of blocks to be invalidated is below
the full scan threshold. This improves the DropRelationFiles()
performance, for example, when the TRUNCATE command truncated off any
of the empty pages at the end of relation.

As a limitation, the optimization does not work if there are TOAST
relations that use compression and/or out-of-line storage.  On the
other hand, using non-TOASTable data types or PLAIN strategy allows
us to enter the optimization path.
---
 src/backend/storage/buffer/bufmgr.c | 89 ++++++++++++++++++++++++++++++++-----
 src/backend/storage/smgr/smgr.c     | 14 +++---
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 87 insertions(+), 18 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index c3ee6c6..a057d02 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3091,28 +3091,30 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
+DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 {
-	int			i,
-				n = 0;
+	int			i;
+	int			j;
+	int			n = 0;
+	BlockNumber (*block)[MAX_FORKNUM + 1];
+	BlockNumber	nBlocksToInvalidate = 0;
 	RelFileNode *nodes;
+	bool		cached = false;
 	bool		use_bsearch;
 
 	if (nnodes == 0)
 		return;
 
-	nodes = palloc(sizeof(RelFileNode) * nnodes);	/* non-local relations */
-
 	/* If it's a local relation, it's localbuf.c's problem. */
 	for (i = 0; i < nnodes; i++)
 	{
-		if (RelFileNodeBackendIsTemp(rnodes[i]))
+		if (RelFileNodeBackendIsTemp(smgr_reln[i]->smgr_rnode))
 		{
-			if (rnodes[i].backend == MyBackendId)
-				DropRelFileNodeAllLocalBuffers(rnodes[i].node);
+			if (smgr_reln[i]->smgr_rnode.backend == MyBackendId)
+				DropRelFileNodeAllLocalBuffers(smgr_reln[i]->smgr_rnode.node);
 		}
 		else
-			nodes[n++] = rnodes[i].node;
+			n++;
 	}
 
 	/*
@@ -3120,11 +3122,78 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	 * memory and return.
 	 */
 	if (n == 0)
+		return;
+
+	block = (BlockNumber (*)[MAX_FORKNUM + 1])
+			palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));
+
+	/*
+	 * Check if all the relations are candidates for buffer drop
+	 * optimization.  Otherwise, if a size of a relation fork is
+	 * not cached, we proceed to full scan of the whole buffer pool.
+	 *
+	 * As a limitation, the optimization does not work if there are
+	 * TOAST relations that use compression and/or out-of-line storage.
+	 * On the other hand, using non-TOASTable data types or PLAIN
+	 * strategy allows us to enter the optimization path.
+	 */
+	for (i = 0; i < n; i++)
 	{
-		pfree(nodes);
+		for (j = 0; j <= MAX_FORKNUM; j++)
+		{
+			/*
+			 * Assign block to InvalidblockNumber if a relation
+			 * fork does not exist, so that we can skip it later
+			 * when dropping the relation buffers.
+			 */
+			if (!smgrexists(smgr_reln[i], j))
+			{
+				block[i][j] = InvalidBlockNumber;
+				continue;
+			}
+
+			/* Get the number of blocks for a relation's fork */
+			block[i][j] = smgrnblocks(smgr_reln[i], j, &cached);
+
+			if (!cached)
+				goto buffer_full_scan;
+
+			nBlocksToInvalidate += block[i][j];
+		}
+	}
+
+	/*
+	 * Enter the optimization if the total number of blocks to be
+	 * invalidated for all relations is below the full scan threshold.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (i = 0; i < n; i++)
+		{
+			/*
+			 * If block to drop is valid, drop the buffers of the fork.
+			 * Zero the firstDelBlock because all buffers will be
+			 * dropped anyway.
+			 */
+			for (j = 0; j <= MAX_FORKNUM; j++)
+			{
+				if (!BlockNumberIsValid(block[i][j]))
+					continue;
+
+				FindAndDropRelFileNodeBuffers(smgr_reln[i]->smgr_rnode.node,
+											  j, block[i][j], 0);
+			}
+		}
+		pfree(block);
 		return;
 	}
 
+buffer_full_scan:
+	pfree(block);
+	nodes = palloc(sizeof(RelFileNode) * n); /* non-local relations */
+	for (i = 0; i < n; i++)
+		nodes[i] = smgr_reln[i]->smgr_rnode.node;
+
 	/*
 	 * For low number of relations to drop just use a simple walk through, to
 	 * save the bsearch overhead. The threshold to use is rather a guess than
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 9d3a67c..3663bb7 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -392,7 +392,13 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 		return;
 
 	/*
-	 * create an array which contains all relations to be dropped, and close
+	 * Get rid of any remaining buffers for the relations.  bufmgr will just
+	 * drop them without bothering to write the contents.
+	 */
+	DropRelFileNodesAllBuffers(rels, nrels);
+
+	/*
+	 * Create an array which contains all relations to be dropped, and close
 	 * each relation's forks at the smgr level while at it
 	 */
 	rnodes = palloc(sizeof(RelFileNodeBackend) * nrels);
@@ -409,12 +415,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	}
 
 	/*
-	 * Get rid of any remaining buffers for the relations.  bufmgr will just
-	 * drop them without bothering to write the contents.
-	 */
-	DropRelFileNodesAllBuffers(rnodes, nrels);
-
-	/*
 	 * It'd be nice to tell the stats collector to forget them immediately,
 	 * too. But we can't because we don't know the OIDs.
 	 */
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 056f65e..2e5189b 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -205,7 +205,7 @@ extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels)
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
-extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
+extern void DropRelFileNodesAllBuffers(struct SMgrRelationData **smgr_reln, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
 #define RelationGetNumberOfBlocks(reln) \
-- 
1.8.3.1

#205tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#204)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

I added comment in 0004 the limitation of optimization when there are TOAST
relations that use NON-PLAIN strategy. i.e. The optimization works if the data
types used are integers, OID, bytea, etc. But for TOAST-able data types like text,
the optimization will be skipped and force a full scan during recovery.

bytea is a TOAST-able type.

+	/*
+	 * Enter the optimization if the total number of blocks to be
+	 * invalidated for all relations is below the full scan threshold.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)

Checking cached here doesn't seem to be necessary, because if cached is false, the control goes to the full scan path as below:

+			if (!cached)
+				goto buffer_full_scan;
+

Regards
Takayuki Tsunakawa

#206Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#204)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Dec 10, 2020 at 1:40 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Yes, I have tested that optimization works for index relations.

I have attached the V34, following the conditions that we use "cached" flag
for both DropRelFileNodesBuffers() and DropRelFileNodesBuffers() for
consistency.
I added comment in 0004 the limitation of optimization when there are TOAST
relations that use NON-PLAIN strategy. i.e. The optimization works if the data
types used are integers, OID, bytea, etc. But for TOAST-able data types like text,
the optimization will be skipped and force a full scan during recovery.

AFAIU, it won't take optimization path only when we have TOAST
relation but there is no insertion corresponding to it. If so, then we
don't need to mention it specifically because there are other similar
cases where the optimization won't work like when during recovery we
have to just perform TRUNCATE.

--
With Regards,
Amit Kapila.

#207k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#206)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thursday, December 10, 2020 8:12 PM, Amit Kapila wrote:

On Thu, Dec 10, 2020 at 1:40 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Yes, I have tested that optimization works for index relations.

I have attached the V34, following the conditions that we use "cached"
flag for both DropRelFileNodesBuffers() and DropRelFileNodesBuffers()
for consistency.
I added comment in 0004 the limitation of optimization when there are
TOAST relations that use NON-PLAIN strategy. i.e. The optimization
works if the data types used are integers, OID, bytea, etc. But for
TOAST-able data types like text, the optimization will be skipped and force a

full scan during recovery.

AFAIU, it won't take optimization path only when we have TOAST relation but
there is no insertion corresponding to it. If so, then we don't need to mention
it specifically because there are other similar cases where the optimization
won't work like when during recovery we have to just perform TRUNCATE.

Right, I forgot to add that there should be an update like insert to the TOAST
relation for truncate optimization to work. However, that is only limited to
TOAST relations with PLAIN strategy. I have tested with text data type, with
Inserts before truncate, and it did not enter the optimization path. OTOH,
It worked for data type like integer. So should I still not include that information?

Also, I will remove the unnecessary "cached" from the line that Tsunakawa-san
mentioned. I will wait for a few more comments before reuploading, hopefully,
the final version & including the test for truncate,

Regards,
Kirk Jamison

#208tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#207)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

On Thursday, December 10, 2020 8:12 PM, Amit Kapila wrote:

AFAIU, it won't take optimization path only when we have TOAST relation but
there is no insertion corresponding to it. If so, then we don't need to mention
it specifically because there are other similar cases where the optimization
won't work like when during recovery we have to just perform TRUNCATE.

Right, I forgot to add that there should be an update like insert to the TOAST
relation for truncate optimization to work. However, that is only limited to
TOAST relations with PLAIN strategy. I have tested with text data type, with
Inserts before truncate, and it did not enter the optimization path. OTOH,
It worked for data type like integer. So should I still not include that information?

What's valuable as a code comment to describe the remaining issue is that the reader can find clues to if this is related to the problem he/she has hit, and/or how to solve the issue. I don't think the current comment is so bad in that regard, but it seems better to add:

* The condition of the issue: the table's ancillary storage (index, TOAST table, FSM, VM, etc.) was not updated during recovery.
(As an aside, "during recovery" here does not mean "after the last checkpoint" but "from the start of recovery", because the standby experiences many checkpoints (the correct term is restartpoints in case of standby).)

* The cause as a hint to solve the issue: The startup process does not find page modification WAL records. As a result, it won't call XLogReadBufferExtended() and smgrnblocks() called therein, so the relation/fork size is not cached.

Regards
Takayuki Tsunakawa

#209tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#208)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: tsunakawa.takay@fujitsu.com <tsunakawa.takay@fujitsu.com>

What's valuable as a code comment to describe the remaining issue is that the

You can attach XXX or FIXME in front of the issue description for easier search. (XXX appears to be used much more often in Postgres.)

Regards
Takayuki Tsunakawa

#210Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#207)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Dec 11, 2020 at 5:54 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Thursday, December 10, 2020 8:12 PM, Amit Kapila wrote:

On Thu, Dec 10, 2020 at 1:40 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Yes, I have tested that optimization works for index relations.

I have attached the V34, following the conditions that we use "cached"
flag for both DropRelFileNodesBuffers() and DropRelFileNodesBuffers()
for consistency.
I added comment in 0004 the limitation of optimization when there are
TOAST relations that use NON-PLAIN strategy. i.e. The optimization
works if the data types used are integers, OID, bytea, etc. But for
TOAST-able data types like text, the optimization will be skipped and force a

full scan during recovery.

AFAIU, it won't take optimization path only when we have TOAST relation but
there is no insertion corresponding to it. If so, then we don't need to mention
it specifically because there are other similar cases where the optimization
won't work like when during recovery we have to just perform TRUNCATE.

Right, I forgot to add that there should be an update like insert to the TOAST
relation for truncate optimization to work. However, that is only limited to
TOAST relations with PLAIN strategy. I have tested with text data type, with
Inserts before truncate, and it did not enter the optimization path.

I think you are seeing because text datatype allows creating toast
storage and your data is small enough to be toasted.

OTOH,
It worked for data type like integer.

It is not related to any datatype, it can happen whenever we don't
have any operation on any of the forks after recovery.

So should I still not include that information?

I think we can extend your existing comment like: "Otherwise if the
size of a relation fork is not cached, we proceed to a full scan of
the whole buffer pool. This can happen if there is no update to a
particular fork during recovery."

--
With Regards,
Amit Kapila.

#211k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#210)
4 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Friday, December 11, 2020 10:27 AM, Amit Kapila wrote:

On Fri, Dec 11, 2020 at 5:54 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

So should I still not include that information?

I think we can extend your existing comment like: "Otherwise if the size of a
relation fork is not cached, we proceed to a full scan of the whole buffer pool.
This can happen if there is no update to a particular fork during recovery."

Attached are the final updated patches.
I followed this advice and updated the source code comment a little bit.
There are no changes from the previous except that and the unnecessary
"cached" condition which Tsunakawa-san mentioned.

Below is also the updated recovery performance test results for TRUNCATE.
(1000 tables, 1MB per table, results measured in seconds)
| s_b | Master | Patched | % Reg |
|-------|--------|---------|---------|
| 128MB | 0.406 | 0.406 | 0% |
| 512MB | 0.506 | 0.406 | -25% |
| 1GB | 0.806 | 0.406 | -99% |
| 20GB | 15.224 | 0.406 | -3650% |
| 100GB | 81.506 | 0.406 | -19975% |

Because of the size of relation, it is expected to enter full-scan for
the 128MB shared_buffers setting. And there was no regression.
Similar to previous test results, the recovery time was constant
for all shared_buffers setting with the patches applied.

Regards,
Kirk Jamison

Attachments:

v35-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v35-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From ae05c52be1e3ecae24441a9d7917a672591f271a Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v35 1/4] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df..f476b8e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -474,7 +474,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v35-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchapplication/octet-stream; name=v35-0002-Add-bool-param-in-smgrnblocks-for-cached-blocks.patchDownload
From e1373927cff0cac3f3a1f6e2bb1aab6f33ec3369 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 7 Oct 2020 09:57:53 +0000
Subject: [PATCH v35 2/4] Add bool param in smgrnblocks() for cached blocks.

The flag ensures that we return a reliable value from smgrnblocks.
---
 src/backend/access/gist/gistbuild.c       |  2 +-
 src/backend/access/heap/visibilitymap.c   |  6 +++---
 src/backend/access/table/tableam.c        |  4 ++--
 src/backend/access/transam/xlogutils.c    |  2 +-
 src/backend/catalog/storage.c             |  4 ++--
 src/backend/storage/buffer/bufmgr.c       |  4 ++--
 src/backend/storage/freespace/freespace.c |  6 +++---
 src/backend/storage/smgr/smgr.c           | 21 ++++++++++++++++++---
 src/include/storage/smgr.h                |  3 ++-
 9 files changed, 34 insertions(+), 18 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 32a3099..afc640d 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -460,7 +460,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index d538f257..3874ff3 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -721,7 +721,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ad0d1a9..1680bf4 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -740,7 +740,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2857,7 +2857,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index f476b8e..e9dffd2 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -535,7 +535,6 @@ smgrwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 										buffer, skipFsync);
 }
 
-
 /*
  *	smgrwriteback() -- Trigger kernel writeback for the supplied range of
  *					   blocks.
@@ -551,18 +550,34 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 /*
  *	smgrnblocks() -- Calculate the number of blocks in the
  *					 supplied relation.
+ *
+ *		The "cached" flag ensures that no buffers exist for blocks after the
+ *		cached value is known to the process during recovery.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *cached)
 {
 	BlockNumber result;
 
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
-	 * invalidation mechanism for changes in file size.
+	 * invalidation mechanism for changes in file size.  In recovery, the cached
+	 * value returned by the first lseek could be smaller than the actual number
+	 * of existing buffers of the file, which is caused by buggy Linux kernels
+	 * that might not have accounted for the recent write.  However, we can still
+	 * rely on the cached value even if we get a bogus value from first lseek
+	 * since it is impossible to have buffer for blocks after that file size.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (cached != NULL)
+			*cached = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
+
+	if (cached != NULL)
+		*cached = false;
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..cd99f1b 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *cached);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v35-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchapplication/octet-stream; name=v35-0003-Optimize-DropRelFileNodeBuffers-during-recovery.patchDownload
From 9a640a89f07116f2b09cc4a3f73f971ee8d6e854 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Tue, 10 Nov 2020 06:27:11 +0000
Subject: [PATCH v35 3/4] Optimize DropRelFileNodeBuffers() during recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool is avoided when the relation
is small enough, or the the total number of blocks to be invalidated
is below the threshold of full scanning. This improves the
performance especially when VACUUM or autovacuum truncated off any of
the empty pages at the end of relation.

While recovery, when WAL files of XLOG_SMGR_TRUNCATE from vacuum or
autovacuum are replayed, we must not leave a buffer for the relations
to be dropped.  So we invalidate buffer blocks by locating using
BufTableLookup() when it is certain that we know up to what page of
every fork we possiblly have a buffer.  We can know that by the
"cached" flag returned by smgrnblocks(), which currently gets true
only while recovery.  It's safe to use the cached nblocks because it
is guaranteed to be the maximum page we have in shared buffer during
recovery, and that guarantee is held by not asking fseek once we have
cached the value.  Also, the cached nblocks will not be invalidated
by file extension during recovery. See smgrnblocks() and smgrextend()
for more details.

If we are not in recovery or when the nblocks is not cached, then we
scan the whole buffer pool to search and drop the buffers of relation,
which is slower when a small part of buffers are to be dropped.
---
 src/backend/storage/buffer/bufmgr.c | 115 +++++++++++++++++++++++++++++++++---
 src/backend/storage/smgr/smgr.c     |   2 +-
 src/include/storage/bufmgr.h        |   2 +-
 3 files changed, 110 insertions(+), 9 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 1680bf4..c3ee6c6 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,8 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32)(NBuffers / 256)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +475,10 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FindAndDropRelFileNodeBuffers(RelFileNode rnode,
+										  ForkNumber forkNum,
+										  BlockNumber nForkBlock,
+										  BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2967,18 +2973,28 @@ BufferGetLSNAtomic(Buffer buffer)
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
  *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
+ *		We must not leave a buffer for the relations to be dropped.  We
+ *		invalidate buffer blocks by locating using BufTableLookup() when it
+ *		is certain that we know up to what page of every fork we possiblly
+ *		have a buffer.  We can know that by the "cached" flag returned by
+ *		smgrnblocks(), which currently gets true only while recovery.  See
+ *		smgrnblocks() and smgrextend().  Otherwise we scan the whole buffer
+ *		pool to find buffers for the relation, which is slower when a small
+ *		part of buffers are to be dropped.
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend	rnode;
+	bool		cached = false;
+	BlockNumber	nForkBlock[MAX_FORKNUM];
+	BlockNumber	nBlocksToInvalidate = 0;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2992,6 +3008,38 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * Get the total number of to-be-invalidated blocks of a relation as
+	 * well as the total blocks for a given fork.  Give up the optimization
+	 * if the block is not cached.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlock[i] = smgrnblocks(smgr_reln, forkNum[i], &cached);
+
+		if (!cached)
+			break;
+
+		/* Get the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlock[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Look up the buffers in the hashtable and drop them if the block size
+	 * is already cached and the total blocks to be invalidated is below the
+	 * full scan threshold.  Otherwise, give up the optimization.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+		{
+			FindAndDropRelFileNodeBuffers(rnode.node, forkNum[j],
+										  nForkBlock[j], firstDelBlock[j]);
+		}
+		return;
+	}
+
 	for (i = 0; i < NBuffers; i++)
 	{
 		BufferDesc *bufHdr = GetBufferDescriptor(i);
@@ -3135,6 +3183,60 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 }
 
 /* ---------------------------------------------------------------------
+ *		FindAndDropRelFileNodeBuffers
+ *
+ *		This function finds and removes from the buffer pool all the pages
+ *		of the specified relation fork that has block number >= firstDelBlock.
+ *		(In particular, with firstDelBlock = 0, all pages are removed.)
+ *		This is only called in recovery when the block count of a fork is
+ *		cached or when the total number of to-be-invalidated blocks for
+ *		involved relation/s does not exceed the threshold for buffer full scan.
+ * --------------------------------------------------------------------
+ */
+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber forkNum,
+							  BlockNumber nForkBlock, BlockNumber firstDelBlock)
+{
+	BlockNumber		curBlock;
+
+	for (curBlock = firstDelBlock; curBlock < nForkBlock; curBlock++)
+	{
+		uint32		bufHash;		/* hash value for tag */
+		BufferTag	bufTag;			/* identity of requested block */
+		LWLock	   	*bufPartitionLock;	/* buffer partition lock for it */
+		int		buf_id;
+		BufferDesc	*bufHdr;
+		uint32		buf_state;
+
+		/* create a tag so we can lookup the buffer */
+		INIT_BUFFERTAG(bufTag, rnode, forkNum, curBlock);
+
+		/* determine its hash code and partition lock ID */
+		bufHash = BufTableHashCode(&bufTag);
+		bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+		/* Check that it is in the buffer pool. If not, do nothing. */
+		LWLockAcquire(bufPartitionLock, LW_SHARED);
+		buf_id = BufTableLookup(&bufTag, bufHash);
+		LWLockRelease(bufPartitionLock);
+
+		if (buf_id < 0)
+			continue;
+
+		bufHdr = GetBufferDescriptor(buf_id);
+
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+/* ---------------------------------------------------------------------
  *		DropDatabaseBuffers
  *
  *		This function removes all the buffers in the buffer cache for a
@@ -3246,8 +3348,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index e9dffd2..9d3a67c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -605,7 +605,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
-- 
1.8.3.1

v35-0004-Optimize-DropRelFileNodesAllBuffers-in-recovery.patchapplication/octet-stream; name=v35-0004-Optimize-DropRelFileNodesAllBuffers-in-recovery.patchDownload
From 9cad80a4a321c88057de82913d3a0ce700982616 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 2 Dec 2020 12:48:04 +0000
Subject: [PATCH v35 4/4] Optimize DropRelFileNodesAllBuffers() in recovery.

DropRelFileNodesAllBuffers() is optimized to skip the time-consuming
scan of the whole buffer pool during recovery when the relation is
small enough, or when the number of blocks to be invalidated is below
the full scan threshold. This improves the DropRelationFiles()
performance, for example, when the TRUNCATE command truncated off any
of the empty pages at the end of relation.

We first check if all the relations are candidates for optimization
during recovery, by ensuring that a cached size was returned by
smgrnblocks() for a relation fork.  Similar to DropRelFileNodeBuffers,
we invalidate buffer blocks by locating using BufTableLookup() when it
is certain that we know up to what page of every fork we possiblly
have a buffer.  That can be checked through the "cached" flag returned
by smgrnblocks(), which currently gets true only while recovery.
Otherwise, we proceed to full scan of the whole buffer pool if a size
of a particular relation fork is not "cached", which can happen if
there are no updates to that existing fork during recovery.
---
 src/backend/storage/buffer/bufmgr.c | 86 ++++++++++++++++++++++++++++++++-----
 src/backend/storage/smgr/smgr.c     | 14 +++---
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 84 insertions(+), 18 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index c3ee6c6..ba5acaf 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3091,28 +3091,30 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
+DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 {
-	int			i,
-				n = 0;
+	int			i;
+	int			j;
+	int			n = 0;
+	BlockNumber (*block)[MAX_FORKNUM + 1];
+	BlockNumber	nBlocksToInvalidate = 0;
 	RelFileNode *nodes;
+	bool		cached = false;
 	bool		use_bsearch;
 
 	if (nnodes == 0)
 		return;
 
-	nodes = palloc(sizeof(RelFileNode) * nnodes);	/* non-local relations */
-
 	/* If it's a local relation, it's localbuf.c's problem. */
 	for (i = 0; i < nnodes; i++)
 	{
-		if (RelFileNodeBackendIsTemp(rnodes[i]))
+		if (RelFileNodeBackendIsTemp(smgr_reln[i]->smgr_rnode))
 		{
-			if (rnodes[i].backend == MyBackendId)
-				DropRelFileNodeAllLocalBuffers(rnodes[i].node);
+			if (smgr_reln[i]->smgr_rnode.backend == MyBackendId)
+				DropRelFileNodeAllLocalBuffers(smgr_reln[i]->smgr_rnode.node);
 		}
 		else
-			nodes[n++] = rnodes[i].node;
+			n++;
 	}
 
 	/*
@@ -3120,11 +3122,75 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	 * memory and return.
 	 */
 	if (n == 0)
+		return;
+
+	block = (BlockNumber (*)[MAX_FORKNUM + 1])
+			palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));
+
+	/*
+	 * Check if all the relations are candidates for buffer drop
+	 * optimization.  Otherwise, we proceed to full scan of the
+	 * whole buffer pool if a size of a particular relation fork
+	 * is not cached, which can happen if there are no updates
+	 * to that fork during recovery.
+	 */
+	for (i = 0; i < n; i++)
 	{
-		pfree(nodes);
+		for (j = 0; j <= MAX_FORKNUM; j++)
+		{
+			/*
+			 * Assign InvalidblockNumber to a block if a relation
+			 * fork does not exist, so that we can skip it later
+			 * when dropping the relation buffers.
+			 */
+			if (!smgrexists(smgr_reln[i], j))
+			{
+				block[i][j] = InvalidBlockNumber;
+				continue;
+			}
+
+			/* Get the number of blocks for a relation's fork */
+			block[i][j] = smgrnblocks(smgr_reln[i], j, &cached);
+
+			if (!cached)
+				goto buffer_full_scan;
+
+			nBlocksToInvalidate += block[i][j];
+		}
+	}
+
+	/*
+	 * Enter the optimization if the total number of blocks to be
+	 * invalidated for all relations is below the full scan threshold.
+	 */
+	if (nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (i = 0; i < n; i++)
+		{
+			/*
+			 * If block to drop is valid, drop the buffers of the fork.
+			 * Zero the firstDelBlock because all buffers will be
+			 * dropped anyway.
+			 */
+			for (j = 0; j <= MAX_FORKNUM; j++)
+			{
+				if (!BlockNumberIsValid(block[i][j]))
+					continue;
+
+				FindAndDropRelFileNodeBuffers(smgr_reln[i]->smgr_rnode.node,
+											  j, block[i][j], 0);
+			}
+		}
+		pfree(block);
 		return;
 	}
 
+buffer_full_scan:
+	pfree(block);
+	nodes = palloc(sizeof(RelFileNode) * n); /* non-local relations */
+	for (i = 0; i < n; i++)
+		nodes[i] = smgr_reln[i]->smgr_rnode.node;
+
 	/*
 	 * For low number of relations to drop just use a simple walk through, to
 	 * save the bsearch overhead. The threshold to use is rather a guess than
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 9d3a67c..3663bb7 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -392,7 +392,13 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 		return;
 
 	/*
-	 * create an array which contains all relations to be dropped, and close
+	 * Get rid of any remaining buffers for the relations.  bufmgr will just
+	 * drop them without bothering to write the contents.
+	 */
+	DropRelFileNodesAllBuffers(rels, nrels);
+
+	/*
+	 * Create an array which contains all relations to be dropped, and close
 	 * each relation's forks at the smgr level while at it
 	 */
 	rnodes = palloc(sizeof(RelFileNodeBackend) * nrels);
@@ -409,12 +415,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	}
 
 	/*
-	 * Get rid of any remaining buffers for the relations.  bufmgr will just
-	 * drop them without bothering to write the contents.
-	 */
-	DropRelFileNodesAllBuffers(rnodes, nrels);
-
-	/*
 	 * It'd be nice to tell the stats collector to forget them immediately,
 	 * too. But we can't because we don't know the OIDs.
 	 */
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 056f65e..2e5189b 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -205,7 +205,7 @@ extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels)
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
-extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
+extern void DropRelFileNodesAllBuffers(struct SMgrRelationData **smgr_reln, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
 #define RelationGetNumberOfBlocks(reln) \
-- 
1.8.3.1

#212tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#211)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>

Attached are the final updated patches.

Looks good, and the patch remains ready for committer. (Personally, I wanted the code comment to touch upon the TOAST and FSM/VM for the reader, because we couldn't think of those possibilities and took some time to find why the optimization path wasn't taken.)

Regards
Takayuki Tsunakawa

#213Tang, Haiying
tanghy.fnst@cn.fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#212)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hello Kirk,

I noticed you have pushed a new version for your patch which has some changes on TRUNCATE on TOAST relation.
Although you've done performance test for your changed part. I'd like to do a double check for your patch(hope you don't mind).
Below is the updated recovery performance test results for your new patch. All seems good.

*TOAST relation with PLAIN strategy like integer :
1. Recovery after VACUUM test results(average of 15 times)
shared_buffers master(sec) patched(sec) %reg=((patched-master)/patched)
--------------------------------------------------------------------------------------
128M 2.111 1.604 -24%
10G 57.135 1.878 -97%
20G 167.122 1.932 -99%

2. Recovery after TRUNCATE test results(average of 15 times)
shared_buffers master(sec) patched(sec) %reg=((patched-master)/patched)
--------------------------------------------------------------------------------------
128M 2.326 1.718 -26%
10G 82.397 1.738 -98%
20G 169.275 1.718 -99%

*TOAST relation with NON-PLAIN strategy like text/varchar:
1. Recovery after VACUUM test results(average of 15 times)
shared_buffers master(sec) patched(sec) %reg=((patched-master)/patched)
--------------------------------------------------------------------------------------
128M 3.174 2.493 -21%
10G 72.716 2.246 -97%
20G 163.660 2.474 -98%

2. Recovery after TRUNCATE test results(average of 15 times): Although it looks like there are some improvements after patch applied. I think that's because of the average calculation. TRUNCATE results should be similar between master and patched because they all do full scan.
shared_buffers master(sec) patched(sec) %reg=((patched-master)/patched)
--------------------------------------------------------------------------------------
128M 4.978 4.958 0%
10G 97.048 88.751 -9%
20G 183.230 173.226 -5%

[Machine spec]
CPU : 40 processors (Intel(R) Xeon(R) Silver 4210 CPU @ 2.20GHz)
Memory: 128G
OS: CentOS 8

[Failover test data]
Total table Size: 600M
Table: 10000 tables (1000 rows per table)

[Configure in postgresql.conf]
autovacuum = off
wal_level = replica
max_wal_senders = 5
max_locks_per_transaction = 10000

If you have any questions on my test results, please let me know.

Regards
Tang

#214Amit Kapila
amit.kapila16@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#169)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Nov 19, 2020 at 12:37 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Andres Freund <andres@anarazel.de>

Smaller comment:

+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber *forkNum,
int nforks,
+                                                       BlockNumber
*nForkBlocks, BlockNumber *firstDelBlock)
...
+                     /* Check that it is in the buffer pool. If not, do nothing.
*/
+                     LWLockAcquire(bufPartitionLock, LW_SHARED);
+                     buf_id = BufTableLookup(&bufTag, bufHash);
...
+                     bufHdr = GetBufferDescriptor(buf_id);
+
+                     buf_state = LockBufHdr(bufHdr);
+
+                     if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+                             bufHdr->tag.forkNum == forkNum[i] &&
+                             bufHdr->tag.blockNum >= firstDelBlock[i])
+                             InvalidateBuffer(bufHdr);       /* releases
spinlock */
+                     else
+                             UnlockBufHdr(bufHdr, buf_state);

I'm a bit confused about the check here. We hold a buffer partition lock, and
have done a lookup in the mapping table. Why are we then rechecking the
relfilenode/fork/blocknum? And why are we doing so holding the buffer header
lock, which is essentially a spinlock, so should only ever be held for very short
portions?

This looks like it's copying logic from DropRelFileNodeBuffers() etc, but there
the situation is different: We haven't done a buffer mapping lookup, and we
don't hold a partition lock!

That's because the buffer partition lock is released immediately after the hash table has been looked up. As an aside, InvalidateBuffer() requires the caller to hold the buffer header spinlock and doesn't hold the buffer partition lock.

This answers the second part of the question but what about the first
part (We hold a buffer partition lock, and have done a lookup in the
mapping table. Why are we then rechecking the
relfilenode/fork/blocknum?)

I think we don't need such a check, rather we can have an Assert
corresponding to that if-condition in the patch. I understand it is
safe to compare relfilenode/fork/blocknum but it might confuse readers
of the code.

I have started doing minor edits to the patch especially planning to
write a theory why is this optimization safe and here is what I can
come up with: "To remove all the pages of the specified relation forks
from the buffer pool, we need to scan the entire buffer pool but we
can optimize it by finding the buffers from BufMapping table provided
we know the exact size of each fork of the relation. The exact size is
required to ensure that we don't leave any buffer for the relation
being dropped as otherwise the background writer or checkpointer can
lead to a PANIC error while flushing buffers corresponding to files
that don't exist.

To know the exact size, we rely on the size cached for each fork by us
during recovery which limits the optimization to recovery and on
standbys but we can easily extend it once we have shared cache for
relation size.

In recovery, we cache the value returned by the first lseek(SEEK_END)
and the future writes keeps the cached value up-to-date. See
smgrextend. It is possible that the value of the first lseek is
smaller than the actual number of existing blocks in the file due to
buggy Linux kernels that might not have accounted for the recent
write. But that should be fine because there must not be any buffers
after that file size.

XXX We would make the extra lseek call for the unoptimized paths but
that is okay because we do it just for the first fork and we anyway
have to scan the entire buffer pool the cost of which is so high that
the extra lseek call won't make any visible difference. However, we
can use InRecovery flag to avoid the additional cost but that doesn't
seem worth it."

Thoughts?

--
With Regards,
Amit Kapila.

#215tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Amit Kapila (#214)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Amit Kapila <amit.kapila16@gmail.com>

This answers the second part of the question but what about the first
part (We hold a buffer partition lock, and have done a lookup in th
mapping table. Why are we then rechecking the
relfilenode/fork/blocknum?)

I think we don't need such a check, rather we can have an Assert
corresponding to that if-condition in the patch. I understand it is
safe to compare relfilenode/fork/blocknum but it might confuse readers
of the code.

Hmm, you're right. I thought someone else could steal the found buffer and use it for another block because the buffer mapping lwlock is released without pinning the buffer or acquiring the buffer header spinlock. However, in this case (replay of TRUNCATE during recovery), nobody steals the buffer: bgwriter or checkpointer doesn't use a buffer for a new block, and the client backend waits for AccessExclusive lock.

I have started doing minor edits to the patch especially planning to
write a theory why is this optimization safe and here is what I can
come up with:

Thank you, that's fluent and easier to understand.

Regards
Takayuki Tsunakawa

#216Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#215)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Tue, 22 Dec 2020 01:42:55 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Amit Kapila <amit.kapila16@gmail.com>

This answers the second part of the question but what about the first
part (We hold a buffer partition lock, and have done a lookup in th
mapping table. Why are we then rechecking the
relfilenode/fork/blocknum?)

I think we don't need such a check, rather we can have an Assert
corresponding to that if-condition in the patch. I understand it is
safe to compare relfilenode/fork/blocknum but it might confuse readers
of the code.

Hmm, you're right. I thought someone else could steal the found
buffer and use it for another block because the buffer mapping
lwlock is released without pinning the buffer or acquiring the
buffer header spinlock. However, in this case (replay of TRUNCATE
during recovery), nobody steals the buffer: bgwriter or checkpointer
doesn't use a buffer for a new block, and the client backend waits
for AccessExclusive lock.

Mmm. If that is true, doesn't the unoptimized path also need the
rechecking?

The AEL doesn't work for a buffer block. No new block can be allocted
for the relation but still BufferAlloc can steal the block for other
relations since the AEL doesn't work for each buffer block. Am I
still missing something?

I have started doing minor edits to the patch especially planning to
write a theory why is this optimization safe and here is what I can
come up with:

Thank you, that's fluent and easier to understand.

+1

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#217Amit Kapila
amit.kapila16@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#215)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Dec 22, 2020 at 7:13 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Amit Kapila <amit.kapila16@gmail.com>

This answers the second part of the question but what about the first
part (We hold a buffer partition lock, and have done a lookup in th
mapping table. Why are we then rechecking the
relfilenode/fork/blocknum?)

I think we don't need such a check, rather we can have an Assert
corresponding to that if-condition in the patch. I understand it is
safe to compare relfilenode/fork/blocknum but it might confuse readers
of the code.

Hmm, you're right. I thought someone else could steal the found buffer and use it for another block because the buffer mapping lwlock is released without pinning the buffer or acquiring the buffer header spinlock.

Okay, I see your point.

However, in this case (replay of TRUNCATE during recovery), nobody steals the buffer: bgwriter or checkpointer doesn't use a buffer for a new block, and the client backend waits for AccessExclusive lock.

Why would all client backends wait for AccessExclusive lock on this
relation? Say, a client needs a buffer for some other relation and
that might evict this buffer after we release the lock on the
partition. In StrategyGetBuffer, it is important to either have a pin
on the buffer or the buffer header itself must be locked to avoid
getting picked as victim buffer. Am I missing something?

--
With Regards,
Amit Kapila.

#218Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#217)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Tue, 22 Dec 2020 08:08:10 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Tue, Dec 22, 2020 at 7:13 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Amit Kapila <amit.kapila16@gmail.com>

This answers the second part of the question but what about the first
part (We hold a buffer partition lock, and have done a lookup in th
mapping table. Why are we then rechecking the
relfilenode/fork/blocknum?)

I think we don't need such a check, rather we can have an Assert
corresponding to that if-condition in the patch. I understand it is
safe to compare relfilenode/fork/blocknum but it might confuse readers
of the code.

Hmm, you're right. I thought someone else could steal the found buffer and use it for another block because the buffer mapping lwlock is released without pinning the buffer or acquiring the buffer header spinlock.

Okay, I see your point.

However, in this case (replay of TRUNCATE during recovery), nobody steals the buffer: bgwriter or checkpointer doesn't use a buffer for a new block, and the client backend waits for AccessExclusive lock.

I understood that you are thinking that the rechecking is useless.

Why would all client backends wait for AccessExclusive lock on this
relation? Say, a client needs a buffer for some other relation and
that might evict this buffer after we release the lock on the
partition. In StrategyGetBuffer, it is important to either have a pin
on the buffer or the buffer header itself must be locked to avoid
getting picked as victim buffer. Am I missing something?

I think exactly like that. If we acquire the bufHdr lock before
releasing the partition lock, that steal doesn't happen but it doesn't
seem good as a locking protocol.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#219tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Amit Kapila (#217)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Amit Kapila <amit.kapila16@gmail.com>

Why would all client backends wait for AccessExclusive lock on this
relation? Say, a client needs a buffer for some other relation and
that might evict this buffer after we release the lock on the
partition. In StrategyGetBuffer, it is important to either have a pin
on the buffer or the buffer header itself must be locked to avoid
getting picked as victim buffer. Am I missing something?

Ouch, right. (The year-end business must be making me crazy...)

So, there are two choices here:

1) The current patch.
2) Acquire the buffer header spinlock before releasing the buffer mapping lwlock, and eliminate the buffer tag comparison as follows:

BufTableLookup();
LockBufHdr();
LWLockRelease();
InvalidateBuffer();

I think both are okay. If I must choose either, I kind of prefer 1), because LWLockRelease() could take longer time to wake up other processes waiting on the lwlock, which is not very good to do while holding a spinlock.

Regards
Takayuki Tsunakawa

#220Amit Kapila
amit.kapila16@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#219)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Dec 22, 2020 at 8:18 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Amit Kapila <amit.kapila16@gmail.com>

Why would all client backends wait for AccessExclusive lock on this
relation? Say, a client needs a buffer for some other relation and
that might evict this buffer after we release the lock on the
partition. In StrategyGetBuffer, it is important to either have a pin
on the buffer or the buffer header itself must be locked to avoid
getting picked as victim buffer. Am I missing something?

Ouch, right. (The year-end business must be making me crazy...)

So, there are two choices here:

1) The current patch.
2) Acquire the buffer header spinlock before releasing the buffer mapping lwlock, and eliminate the buffer tag comparison as follows:

BufTableLookup();
LockBufHdr();
LWLockRelease();
InvalidateBuffer();

I think both are okay. If I must choose either, I kind of prefer 1), because LWLockRelease() could take longer time to wake up other processes waiting on the lwlock, which is not very good to do while holding a spinlock.

I also prefer (1). I will add some comments about the locking protocol
in the next version of the patch.

--
With Regards,
Amit Kapila.

#221Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#218)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Dec 22, 2020 at 8:12 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Tue, 22 Dec 2020 08:08:10 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

Why would all client backends wait for AccessExclusive lock on this
relation? Say, a client needs a buffer for some other relation and
that might evict this buffer after we release the lock on the
partition. In StrategyGetBuffer, it is important to either have a pin
on the buffer or the buffer header itself must be locked to avoid
getting picked as victim buffer. Am I missing something?

I think exactly like that. If we acquire the bufHdr lock before
releasing the partition lock, that steal doesn't happen but it doesn't
seem good as a locking protocol.

Right, so let's keep the code as it is but I feel it is better to add
some comments explaining the rationale behind this code.

--
With Regards,
Amit Kapila.

#222Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#219)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Tue, 22 Dec 2020 02:48:22 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Amit Kapila <amit.kapila16@gmail.com>

Why would all client backends wait for AccessExclusive lock on this
relation? Say, a client needs a buffer for some other relation and
that might evict this buffer after we release the lock on the
partition. In StrategyGetBuffer, it is important to either have a pin
on the buffer or the buffer header itself must be locked to avoid
getting picked as victim buffer. Am I missing something?

Ouch, right. (The year-end business must be making me crazy...)

So, there are two choices here:

1) The current patch.
2) Acquire the buffer header spinlock before releasing the buffer mapping lwlock, and eliminate the buffer tag comparison as follows:

BufTableLookup();
LockBufHdr();
LWLockRelease();
InvalidateBuffer();

I think both are okay. If I must choose either, I kind of prefer 1), because LWLockRelease() could take longer time to wake up other processes waiting on the lwlock, which is not very good to do while holding a spinlock.

I like, as said before, the current patch.

regareds.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#223tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Kyotaro Horiguchi (#216)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

Mmm. If that is true, doesn't the unoptimized path also need the
rechecking?

Yes, the traditional processing does the recheck after acquiring the buffer header spinlock.

Regards
Takayuki Tsunakawa

#224k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#214)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Monday, December 21, 2020 10:25 PM, Amit Kapila wrote:

I have started doing minor edits to the patch especially planning to write a
theory why is this optimization safe and here is what I can come up with:
"To
remove all the pages of the specified relation forks from the buffer pool, we
need to scan the entire buffer pool but we can optimize it by finding the
buffers from BufMapping table provided we know the exact size of each fork
of the relation. The exact size is required to ensure that we don't leave any
buffer for the relation being dropped as otherwise the background writer or
checkpointer can lead to a PANIC error while flushing buffers corresponding
to files that don't exist.

To know the exact size, we rely on the size cached for each fork by us during
recovery which limits the optimization to recovery and on standbys but we
can easily extend it once we have shared cache for relation size.

In recovery, we cache the value returned by the first lseek(SEEK_END) and
the future writes keeps the cached value up-to-date. See smgrextend. It is
possible that the value of the first lseek is smaller than the actual number of
existing blocks in the file due to buggy Linux kernels that might not have
accounted for the recent write. But that should be fine because there must
not be any buffers after that file size.

XXX We would make the extra lseek call for the unoptimized paths but that is
okay because we do it just for the first fork and we anyway have to scan the
entire buffer pool the cost of which is so high that the extra lseek call won't
make any visible difference. However, we can use InRecovery flag to avoid the
additional cost but that doesn't seem worth it."

Thoughts?

+1
Thank you very much for expanding the comments to carefully explain the
reason on why the optimization is safe. I was also struggling to explain it completely
but your description also covers the possibility of extending the optimization in the
future once we have shared cache for rel size. So I like this addition.

(Also, it seems that we have concluded to retain the locking mechanism of the
existing patch based from the recent email exchanges. Both the traditional path and
the optimized path do the rechecking. So there seems to be no problem, I'm definitely
fine with it.)

Regards,
Kirk Jamison

#225Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#222)
1 attachment(s)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Dec 22, 2020 at 8:30 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Tue, 22 Dec 2020 02:48:22 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Amit Kapila <amit.kapila16@gmail.com>

Why would all client backends wait for AccessExclusive lock on this
relation? Say, a client needs a buffer for some other relation and
that might evict this buffer after we release the lock on the
partition. In StrategyGetBuffer, it is important to either have a pin
on the buffer or the buffer header itself must be locked to avoid
getting picked as victim buffer. Am I missing something?

Ouch, right. (The year-end business must be making me crazy...)

So, there are two choices here:

1) The current patch.
2) Acquire the buffer header spinlock before releasing the buffer mapping lwlock, and eliminate the buffer tag comparison as follows:

BufTableLookup();
LockBufHdr();
LWLockRelease();
InvalidateBuffer();

I think both are okay. If I must choose either, I kind of prefer 1), because LWLockRelease() could take longer time to wake up other processes waiting on the lwlock, which is not very good to do while holding a spinlock.

I like, as said before, the current patch.

Attached, please find the updated patch with the following
modifications, (a) updated comments at various places especially to
tell why this is a safe optimization, (b) merged the patch for
extending the smgrnblocks and vacuum optimization patch, (c) made
minor cosmetic changes and ran pgindent, and (d) updated commit
message. BTW, this optimization will help not only vacuum but also
truncate when it is done in the same transaction in which the relation
is created. I would like to see certain tests to ensure that the
value we choose for BUF_DROP_FULL_SCAN_THRESHOLD is correct. I see
that some testing has been done earlier [1]/messages/by-id/OSBPR01MB234176B1829AECFE9FDDFCC2EFE90@OSBPR01MB2341.jpnprd01.prod.outlook.com for this threshold but I
am not still able to conclude. The criteria to find the right
threshold should be what is the maximum size of relation to be
truncated above which we don't get benefit with this optimization.

One idea could be to remove "nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD" part of check "if (cached &&
nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)" so that it always
use optimized path for the tests. Then use the relation size as
NBuffers/128, NBuffers/256, NBuffers/512 for different values of
shared buffers as 128MB, 1GB, 20GB, 100GB.

Apart from tests, do let me know if you are happy with the changes in
the patch? Next, I'll look into DropRelFileNodesAllBuffers()
optimization patch.

[1]: /messages/by-id/OSBPR01MB234176B1829AECFE9FDDFCC2EFE90@OSBPR01MB2341.jpnprd01.prod.outlook.com

--
With Regards,
Amit Kapila.

Attachments:

v36-0001-Optimize-DropRelFileNodeBuffers-for-recovery.patchapplication/octet-stream; name=v36-0001-Optimize-DropRelFileNodeBuffers-for-recovery.patchDownload
From 93d8b04e1e8c71110f66e34b45b672560f75112d Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Tue, 22 Dec 2020 12:04:18 +0530
Subject: [PATCH v36] Optimize DropRelFileNodeBuffers() for recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool can be avoided when the number of blocks
to be truncated in a relation is below a certain threshold. For such cases,
we find the buffers by doing lookups in BufMapping table. This improves
the performance by more than 100 times in many cases for small tables
where the server is configured with a large value of shared buffers
(greater than 20GB).

This optimization helps cases (a) when vacuum or autovacuum truncated off
any of the empty pages at the end of a relation, or (b) when the relation is
truncated in the same transaction in which it was created.

This commit extends the smgrnblocks interface to return a boolean value
which indicates that the cached value for the number of blocks is returned.
This helps us to determine the exact size of relation which is required to
apply this optimization. The exact size is required to ensure that we
don't leave any buffer for the relation being dropped as otherwise the
background writer or checkpointer can lead to a PANIC error while flushing
buffers corresponding to files that don't exist.

Author: Kirk Jamison
Reviewed-by: Kyotaro Horiguchi, Takayuki Tsunakawa, and Amit Kapila
Discussion: https://postgr.es/m/OSBPR01MB3207DCA7EC725FDD661B3EDAEF660@OSBPR01MB3207.jpnprd01.prod.outlook.com
---
 src/backend/access/gist/gistbuild.c       |   2 +-
 src/backend/access/heap/visibilitymap.c   |   6 +-
 src/backend/access/table/tableam.c        |   4 +-
 src/backend/access/transam/xlogutils.c    |   2 +-
 src/backend/catalog/storage.c             |   4 +-
 src/backend/storage/buffer/bufmgr.c       | 144 +++++++++++++++++++++++++++---
 src/backend/storage/freespace/freespace.c |   6 +-
 src/backend/storage/smgr/smgr.c           |  15 +++-
 src/include/storage/bufmgr.h              |   2 +-
 src/include/storage/smgr.h                |   3 +-
 10 files changed, 162 insertions(+), 26 deletions(-)

diff --git a/src/backend/access/gist/gistbuild.c b/src/backend/access/gist/gistbuild.c
index 9d3fa9c..d4a3bff 100644
--- a/src/backend/access/gist/gistbuild.c
+++ b/src/backend/access/gist/gistbuild.c
@@ -860,7 +860,7 @@ gistBuildCallback(Relation index,
 	 */
 	if ((buildstate->buildMode == GIST_BUFFERING_AUTO &&
 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
-		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM, NULL)) ||
 		(buildstate->buildMode == GIST_BUFFERING_STATS &&
 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
 	{
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b107218..ef8533e 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -528,7 +528,7 @@ visibilitymap_prepare_truncate(Relation rel, BlockNumber nheapblocks)
 	else
 		newnblocks = truncBlock;
 
-	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
+	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
 		return InvalidBlockNumber;
@@ -564,7 +564,7 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
 	{
 		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
 	}
@@ -647,7 +647,7 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
-	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
+	vm_nblocks_now = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM, NULL);
 
 	/* Now extend the file */
 	while (vm_nblocks_now < vm_nblocks)
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 6438c45..75eade8 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -636,10 +636,10 @@ table_block_relation_size(Relation rel, ForkNumber forkNumber)
 	if (forkNumber == InvalidForkNumber)
 	{
 		for (int i = 0; i < MAX_FORKNUM; i++)
-			nblocks += smgrnblocks(rel->rd_smgr, i);
+			nblocks += smgrnblocks(rel->rd_smgr, i, NULL);
 	}
 	else
-		nblocks = smgrnblocks(rel->rd_smgr, forkNumber);
+		nblocks = smgrnblocks(rel->rd_smgr, forkNumber, NULL);
 
 	return nblocks * BLCKSZ;
 }
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index e0ca385..f4c0e01 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -459,7 +459,7 @@ XLogReadBufferExtended(RelFileNode rnode, ForkNumber forknum,
 	 */
 	smgrcreate(smgr, forknum, true);
 
-	lastblock = smgrnblocks(smgr, forknum);
+	lastblock = smgrnblocks(smgr, forknum, NULL);
 
 	if (blkno < lastblock)
 	{
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index d538f257..3874ff3 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -434,7 +434,7 @@ RelationCopyStorage(SMgrRelation src, SMgrRelation dst,
 	use_wal = XLogIsNeeded() &&
 		(relpersistence == RELPERSISTENCE_PERMANENT || copying_initfork);
 
-	nblocks = smgrnblocks(src, forkNum);
+	nblocks = smgrnblocks(src, forkNum, NULL);
 
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
@@ -721,7 +721,7 @@ smgrDoPendingSyncs(bool isCommit, bool isParallelWorker)
 			{
 				if (smgrexists(srel, fork))
 				{
-					BlockNumber n = smgrnblocks(srel, fork);
+					BlockNumber n = smgrnblocks(srel, fork, NULL);
 
 					/* we shouldn't come here for unlogged relations */
 					Assert(fork != INIT_FORKNUM);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index c5e8707..984a8e1 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,14 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+/*
+ * This is the size (in the number of blocks) above which we scan the
+ * entire buffer pool to remove the buffers for all the pages of relation
+ * being dropped. For the relations with size below this threshold, we find
+ * the buffers by doing lookups in BufMapping table.
+ */
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32) (NBuffers / 256)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +481,10 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FindAndDropRelFileNodeBuffers(RelFileNode rnode,
+										  ForkNumber forkNum,
+										  BlockNumber nForkBlock,
+										  BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -740,7 +752,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	/* Substitute proper block number if caller asked for P_NEW */
 	if (isExtend)
-		blockNum = smgrnblocks(smgr, forkNum);
+		blockNum = smgrnblocks(smgr, forkNum, NULL);
 
 	if (isLocalBuf)
 	{
@@ -2856,7 +2868,7 @@ RelationGetNumberOfBlocksInFork(Relation relation, ForkNumber forkNum)
 			/* Open it at the smgr level if not already done */
 			RelationOpenSmgr(relation);
 
-			return smgrnblocks(relation->rd_smgr, forkNum);
+			return smgrnblocks(relation->rd_smgr, forkNum, NULL);
 
 		case RELKIND_RELATION:
 		case RELKIND_TOASTVALUE:
@@ -2965,19 +2977,20 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend rnode;
+	bool		cached = false;
+	BlockNumber nForkBlock[MAX_FORKNUM];
+	BlockNumber nBlocksToInvalidate = 0;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2991,6 +3004,59 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * To remove all the pages of the specified relation forks from the buffer
+	 * pool, we need to scan the entire buffer pool but we can optimize it by
+	 * finding the buffers from BufMapping table provided we know the exact
+	 * size of each fork of the relation. The exact size is required to ensure
+	 * that we don't leave any buffer for the relation being dropped as
+	 * otherwise the background writer or checkpointer can lead to a PANIC
+	 * error while flushing buffers corresponding to files that don't exist.
+	 *
+	 * To know the exact size, we rely on the size cached for each fork by us
+	 * during recovery which limits the optimization to recovery and on
+	 * standbys but we can easily extend it once we have shared cache for
+	 * relation size.
+	 *
+	 * In recovery, we cache the value returned by the first lseek(SEEK_END)
+	 * and the future writes keeps the cached value up-to-date. See
+	 * smgrextend. It is possible that the value of the first lseek is smaller
+	 * than the actual number of existing blocks in the file due to buggy
+	 * Linux kernels that might not have accounted for the recent write. But
+	 * that should be fine because there must not be any buffers after that
+	 * file size.
+	 *
+	 * XXX We would make the extra lseek call for the unoptimized paths but
+	 * that is okay because we do it just for the first fork and we anyway
+	 * have to scan the entire buffer pool the cost of which is so high that
+	 * the extra lseek call won't make any visible difference. However, we can
+	 * use InRecovery flag to avoid the additional cost but that doesn't seem
+	 * worth it.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlock[i] = smgrnblocks(smgr_reln, forkNum[i], &cached);
+
+		if (!cached)
+			break;
+
+		/* Get the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlock[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * We apply the optimization iff the total number of blocks to invalidate
+	 * is below the BUF_DROP_FULL_SCAN_THRESHOLD.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+			FindAndDropRelFileNodeBuffers(rnode.node, forkNum[j],
+										  nForkBlock[j], firstDelBlock[j]);
+		return;
+	}
+
 	for (i = 0; i < NBuffers; i++)
 	{
 		BufferDesc *bufHdr = GetBufferDescriptor(i);
@@ -3134,6 +3200,65 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 }
 
 /* ---------------------------------------------------------------------
+ *		FindAndDropRelFileNodeBuffers
+ *
+ *		This function performs look up in BufMapping table and removes from the
+ *		buffer pool all the pages of the specified relation fork that has block
+ *		number >= firstDelBlock. (In particular, with firstDelBlock = 0, all
+ *		pages are removed.)
+ * --------------------------------------------------------------------
+ */
+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber forkNum,
+							  BlockNumber nForkBlock,
+							  BlockNumber firstDelBlock)
+{
+	BlockNumber curBlock;
+
+	for (curBlock = firstDelBlock; curBlock < nForkBlock; curBlock++)
+	{
+		uint32		bufHash;	/* hash value for tag */
+		BufferTag	bufTag;		/* identity of requested block */
+		LWLock	   *bufPartitionLock;	/* buffer partition lock for it */
+		int			buf_id;
+		BufferDesc *bufHdr;
+		uint32		buf_state;
+
+		/* create a tag so we can lookup the buffer */
+		INIT_BUFFERTAG(bufTag, rnode, forkNum, curBlock);
+
+		/* determine its hash code and partition lock ID */
+		bufHash = BufTableHashCode(&bufTag);
+		bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+		/* Check that it is in the buffer pool. If not, do nothing. */
+		LWLockAcquire(bufPartitionLock, LW_SHARED);
+		buf_id = BufTableLookup(&bufTag, bufHash);
+		LWLockRelease(bufPartitionLock);
+
+		if (buf_id < 0)
+			continue;
+
+		bufHdr = GetBufferDescriptor(buf_id);
+
+		/*
+		 * We need to lock the buffer header and recheck if the buffer is
+		 * still associated with the same block because the buffer could be
+		 * evicted by some other backend loading blocks for a different
+		 * relation after we release lock on the BufMapping table.
+		 */
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+/* ---------------------------------------------------------------------
  *		DropDatabaseBuffers
  *
  *		This function removes all the buffers in the buffer cache for a
@@ -3245,8 +3370,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126..2ac802c 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -317,7 +317,7 @@ FreeSpaceMapPrepareTruncateRel(Relation rel, BlockNumber nblocks)
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
-		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
+		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL) <= new_nfsmblocks)
 			return InvalidBlockNumber;	/* nothing to do; the FSM was already
 										 * smaller */
 	}
@@ -547,7 +547,7 @@ fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 		/* Invalidate the cache so smgrnblocks asks the kernel. */
 		rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
 		if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
-			smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+			smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 		else
 			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
 	}
@@ -633,7 +633,7 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 
 	/* Invalidate cache so that smgrnblocks() asks the kernel. */
 	rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = InvalidBlockNumber;
-	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	fsm_nblocks_now = smgrnblocks(rel->rd_smgr, FSM_FORKNUM, NULL);
 
 	while (fsm_nblocks_now < fsm_nblocks)
 	{
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 072bdd1..360c78e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -543,9 +543,12 @@ smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 /*
  *	smgrnblocks() -- Calculate the number of blocks in the
  *					 supplied relation.
+ *
+ * The cached flag indicates that the cached value for the number of blocks
+ * is returned. We set this only if requested by the caller.
  */
 BlockNumber
-smgrnblocks(SMgrRelation reln, ForkNumber forknum)
+smgrnblocks(SMgrRelation reln, ForkNumber forknum, bool *cached)
 {
 	BlockNumber result;
 
@@ -554,7 +557,15 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 	 * invalidation mechanism for changes in file size.
 	 */
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+	{
+		if (cached)
+			*cached = true;
+
 		return reln->smgr_cached_nblocks[forknum];
+	}
+
+	if (cached)
+		*cached = false;
 
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
@@ -582,7 +593,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..cd99f1b 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -98,7 +98,8 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 					  BlockNumber blocknum, char *buffer, bool skipFsync);
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
-extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum,
+							   bool *cached);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

#226Amit Kapila
amit.kapila16@gmail.com
In reply to: Amit Kapila (#225)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Dec 22, 2020 at 2:55 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

Apart from tests, do let me know if you are happy with the changes in
the patch? Next, I'll look into DropRelFileNodesAllBuffers()
optimization patch.

Review of v35-0004-Optimize-DropRelFileNodesAllBuffers-in-recovery [1]
========================================================
1.
DropRelFileNodesAllBuffers()
{
..
+buffer_full_scan:
+ pfree(block);
+ nodes = palloc(sizeof(RelFileNode) * n); /* non-local relations */
+ for (i = 0; i < n; i++)
+ nodes[i] = smgr_reln[i]->smgr_rnode.node;
+
..
}

How is it correct to assign nodes array directly from smgr_reln? There
is no one-to-one correspondence. If you see the code before patch, the
passed array can have mixed of temp and non-temp relation information.

2.
+ for (i = 0; i < n; i++)
  {
- pfree(nodes);
+ for (j = 0; j <= MAX_FORKNUM; j++)
+ {
+ /*
+ * Assign InvalidblockNumber to a block if a relation
+ * fork does not exist, so that we can skip it later
+ * when dropping the relation buffers.
+ */
+ if (!smgrexists(smgr_reln[i], j))
+ {
+ block[i][j] = InvalidBlockNumber;
+ continue;
+ }
+
+ /* Get the number of blocks for a relation's fork */
+ block[i][j] = smgrnblocks(smgr_reln[i], j, &cached);

Similar to above, how can we assume smgr_reln array has all non-local
relations? Have we tried the case with mix of temp and non-temp
relations?

In this code, I am slightly worried about the additional cost of each
time checking smgrexists. Consider a case where there are many
relations and only one or few of them have not cached the information,
in such a case we will pay the cost of smgrexists for many relations
without even going to the optimized path. Can we avoid that in some
way or at least reduce its usage to only when it is required? One idea
could be that we first check if the nblocks information is cached and
if so then we don't need to call smgrnblocks, otherwise, check if it
exists. For this, we need an API like smgrnblocks_cahced, something we
discussed earlier but preferred the current API. Do you have any
better ideas?

[1]: /messages/by-id/OSBPR01MB2341882F416A282C3F7D769DEFC70@OSBPR01MB2341.jpnprd01.prod.outlook.com

--
With Regards,
Amit Kapila.

#227k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#225)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Tuesday, December 22, 2020 6:25 PM, Amit Kapila wrote:

Attached, please find the updated patch with the following modifications, (a)
updated comments at various places especially to tell why this is a safe
optimization, (b) merged the patch for extending the smgrnblocks and
vacuum optimization patch, (c) made minor cosmetic changes and ran
pgindent, and (d) updated commit message. BTW, this optimization will help
not only vacuum but also truncate when it is done in the same transaction in
which the relation is created. I would like to see certain tests to ensure that
the value we choose for BUF_DROP_FULL_SCAN_THRESHOLD is correct. I
see that some testing has been done earlier [1] for this threshold but I am not
still able to conclude. The criteria to find the right threshold should be what is
the maximum size of relation to be truncated above which we don't get
benefit with this optimization.

One idea could be to remove "nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD" part of check "if (cached &&
nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)" so that it
always use optimized path for the tests. Then use the relation size as
NBuffers/128, NBuffers/256, NBuffers/512 for different values of shared
buffers as 128MB, 1GB, 20GB, 100GB.

Alright. I will also repeat the tests with the different threshold settings,
and thank you for the tip.

Apart from tests, do let me know if you are happy with the changes in the
patch? Next, I'll look into DropRelFileNodesAllBuffers() optimization patch.

Thank you, Amit.
That looks more neat, combining the previous patches 0002-0003, so I am +1
with the changes because of the clearer explanations for the threshold and
optimization path in DropRelFileNodeBuffers. Thanks for cleaning my patch sets.
Hope we don't forget the 0001 patch's assertion in smgrextend() to ensure that we
do it safely too and that we are not InRecovery.

[1] -
/messages/by-id/OSBPR01MB234176B1829AECFE9
FDDFCC2EFE90%40OSBPR01MB2341.jpnprd01.prod.outlook.com

Regards,
Kirk Jamison

#228Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#227)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Dec 23, 2020 at 6:30 AM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Tuesday, December 22, 2020 6:25 PM, Amit Kapila wrote:

Apart from tests, do let me know if you are happy with the changes in the
patch? Next, I'll look into DropRelFileNodesAllBuffers() optimization patch.

Thank you, Amit.
That looks more neat, combining the previous patches 0002-0003, so I am +1
with the changes because of the clearer explanations for the threshold and
optimization path in DropRelFileNodeBuffers. Thanks for cleaning my patch sets.
Hope we don't forget the 0001 patch's assertion in smgrextend() to ensure that we
do it safely too and that we are not InRecovery.

I think the 0001 is mostly for test purposes but we will see once the
main patches are ready.

--
With Regards,
Amit Kapila.

#229Amit Kapila
amit.kapila16@gmail.com
In reply to: Amit Kapila (#226)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Tue, Dec 22, 2020 at 5:41 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Tue, Dec 22, 2020 at 2:55 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

Apart from tests, do let me know if you are happy with the changes in
the patch? Next, I'll look into DropRelFileNodesAllBuffers()
optimization patch.

Review of v35-0004-Optimize-DropRelFileNodesAllBuffers-in-recovery [1]
========================================================

In this code, I am slightly worried about the additional cost of each
time checking smgrexists. Consider a case where there are many
relations and only one or few of them have not cached the information,
in such a case we will pay the cost of smgrexists for many relations
without even going to the optimized path. Can we avoid that in some
way or at least reduce its usage to only when it is required? One idea
could be that we first check if the nblocks information is cached and
if so then we don't need to call smgrnblocks, otherwise, check if it
exists. For this, we need an API like smgrnblocks_cahced, something we
discussed earlier but preferred the current API. Do you have any
better ideas?

One more idea which is not better than what I mentioned above is that
we completely avoid calling smgrexists and rely on smgrnblocks. It
will throw an error in case the particular fork doesn't exist and we
can use try .. catch to handle it. I just mentioned it as it came
across my mind but I don't think it is better than the previous one.

One more thing about patch:
+ /* Get the number of blocks for a relation's fork */
+ block[i][j] = smgrnblocks(smgr_reln[i], j, &cached);
+
+ if (!cached)
+ goto buffer_full_scan;

Why do we need to use goto here? We can simply break from the loop and
then check if (cached && nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD). I think we should try to avoid goto if
possible without much complexity.

--
With Regards,
Amit Kapila.

#230tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Amit Kapila (#229)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Amit Kapila <amit.kapila16@gmail.com>

+ /* Get the number of blocks for a relation's fork */
+ block[i][j] = smgrnblocks(smgr_reln[i], j, &cached);
+
+ if (!cached)
+ goto buffer_full_scan;

Why do we need to use goto here? We can simply break from the loop and
then check if (cached && nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD). I think we should try to avoid goto if
possible without much complexity.

That's because two for loops are nested -- breaking there only exits the inner loop. (I thought the same as you at first... And I understand some people have alergy to goto, I think modest use of goto makes the code readable.)

Regards
Takayuki Tsunakawa

#231Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#230)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Wed, 23 Dec 2020 04:22:19 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Amit Kapila <amit.kapila16@gmail.com>

+ /* Get the number of blocks for a relation's fork */
+ block[i][j] = smgrnblocks(smgr_reln[i], j, &cached);
+
+ if (!cached)
+ goto buffer_full_scan;

Why do we need to use goto here? We can simply break from the loop and
then check if (cached && nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD). I think we should try to avoid goto if
possible without much complexity.

That's because two for loops are nested -- breaking there only exits the inner loop. (I thought the same as you at first... And I understand some people have alergy to goto, I think modest use of goto makes the code readable.)

I don't strongly oppose to goto's but in this case the outer loop can
break on the same condition with the inner loop, since cached is true
whenever the inner loop runs to the end. It is needed to initialize
the variable cache with true, instead of false, though.

The same pattern is seen in the tree.

Regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#232Zhihong Yu
zyu@yugabyte.com
In reply to: tsunakawa.takay@fujitsu.com (#230)
Re: [Patch] Optimize dropping of relation buffers using dlist

Hi,
It is possible to come out of the nested loop without goto.

+   bool        cached = true;
...
+    * to that fork during recovery.
+    */
+   for (i = 0; i < n && cached; i++)
...
+           if (!cached)
+.              break;

Here I changed the initial value for cached to true so that we enter the
outer loop.
In place of the original goto, we break out of inner loop and exit outer
loop.

Cheers

On Tue, Dec 22, 2020 at 8:22 PM tsunakawa.takay@fujitsu.com <
tsunakawa.takay@fujitsu.com> wrote:

Show quoted text

From: Amit Kapila <amit.kapila16@gmail.com>

+ /* Get the number of blocks for a relation's fork */
+ block[i][j] = smgrnblocks(smgr_reln[i], j, &cached);
+
+ if (!cached)
+ goto buffer_full_scan;

Why do we need to use goto here? We can simply break from the loop and
then check if (cached && nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD). I think we should try to avoid goto if
possible without much complexity.

That's because two for loops are nested -- breaking there only exits the
inner loop. (I thought the same as you at first... And I understand some
people have alergy to goto, I think modest use of goto makes the code
readable.)

Regards
Takayuki Tsunakawa

#233k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#226)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Tuesday, December 22, 2020 9:11 PM, Amit Kapila wrote:

On Tue, Dec 22, 2020 at 2:55 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:

Next, I'll look into DropRelFileNodesAllBuffers()
optimization patch.

Review of v35-0004-Optimize-DropRelFileNodesAllBuffers-in-recovery [1]
=================================================
=======
1.
DropRelFileNodesAllBuffers()
{
..
+buffer_full_scan:
+ pfree(block);
+ nodes = palloc(sizeof(RelFileNode) * n); /* non-local relations */
+for (i = 0; i < n; i++)  nodes[i] = smgr_reln[i]->smgr_rnode.node;
+
..
}

How is it correct to assign nodes array directly from smgr_reln? There is no
one-to-one correspondence. If you see the code before patch, the passed
array can have mixed of temp and non-temp relation information.

You are right. I mistakenly removed the array node that should have been
allocated for non-local relations. So I fixed that by doing:

SMgrRelation *rels;

rels = palloc(sizeof(SMgrRelation) * nnodes); /* non-local relations */

/* If it's a local relation, it's localbuf.c's problem. */
for (i = 0; i < nnodes; i++)
{
if (RelFileNodeBackendIsTemp(smgr_reln[i]->smgr_rnode))
{
if (smgr_reln[i]->smgr_rnode.backend == MyBackendId)
DropRelFileNodeAllLocalBuffers(smgr_reln[i]->smgr_rnode.node);
}
else
rels[n++] = smgr_reln[i];
}
...
if (n == 0)
{
pfree(rels);
return;
}
...
//traditional path:

pfree(block);
nodes = palloc(sizeof(RelFileNode) * n); /* non-local relations */
for (i = 0; i < n; i++)
nodes[i] = rels[i]->smgr_rnode.node;

2.
+ for (i = 0; i < n; i++)
{
- pfree(nodes);
+ for (j = 0; j <= MAX_FORKNUM; j++)
+ {
+ /*
+ * Assign InvalidblockNumber to a block if a relation
+ * fork does not exist, so that we can skip it later
+ * when dropping the relation buffers.
+ */
+ if (!smgrexists(smgr_reln[i], j))
+ {
+ block[i][j] = InvalidBlockNumber;
+ continue;
+ }
+
+ /* Get the number of blocks for a relation's fork */ block[i][j] =
+ smgrnblocks(smgr_reln[i], j, &cached);

Similar to above, how can we assume smgr_reln array has all non-local
relations? Have we tried the case with mix of temp and non-temp relations?

Similar to above reply.

In this code, I am slightly worried about the additional cost of each time
checking smgrexists. Consider a case where there are many relations and only
one or few of them have not cached the information, in such a case we will
pay the cost of smgrexists for many relations without even going to the
optimized path. Can we avoid that in some way or at least reduce its usage to
only when it is required? One idea could be that we first check if the nblocks
information is cached and if so then we don't need to call smgrnblocks,
otherwise, check if it exists. For this, we need an API like
smgrnblocks_cahced, something we discussed earlier but preferred the
current API. Do you have any better ideas?

Right. I understand the point that let's say there are 100 relations, and
the first 99 non-local relations happen to enter the optimization path, but the last
one does not, calling smgrexist() would be too costly and waste of time in that case.
The only solution I could think of as you mentioned is to reintroduce the new API
which we discussed before: smgrnblocks_cached().
It's possible that we call smgrexist() only if smgrnblocks_cached() returns
InvalidBlockNumber.
So if everyone agrees, we can add that API smgrnblocks_cached() which will
Include the "cached" flag parameter, and remove the additional flag modifications
from smgrnblocks(). In this case, both DropRelFileNodeBuffers() and
DropRelFileNodesAllBuffers() will use the new API.

Thoughts?

Regards,
Kirk Jamison

#234Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#233)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Dec 23, 2020 at 1:07 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

On Tuesday, December 22, 2020 9:11 PM, Amit Kapila wrote:

In this code, I am slightly worried about the additional cost of each time
checking smgrexists. Consider a case where there are many relations and only
one or few of them have not cached the information, in such a case we will
pay the cost of smgrexists for many relations without even going to the
optimized path. Can we avoid that in some way or at least reduce its usage to
only when it is required? One idea could be that we first check if the nblocks
information is cached and if so then we don't need to call smgrnblocks,
otherwise, check if it exists. For this, we need an API like
smgrnblocks_cahced, something we discussed earlier but preferred the
current API. Do you have any better ideas?

Right. I understand the point that let's say there are 100 relations, and
the first 99 non-local relations happen to enter the optimization path, but the last
one does not, calling smgrexist() would be too costly and waste of time in that case.
The only solution I could think of as you mentioned is to reintroduce the new API
which we discussed before: smgrnblocks_cached().
It's possible that we call smgrexist() only if smgrnblocks_cached() returns
InvalidBlockNumber.
So if everyone agrees, we can add that API smgrnblocks_cached() which will
Include the "cached" flag parameter, and remove the additional flag modifications
from smgrnblocks(). In this case, both DropRelFileNodeBuffers() and
DropRelFileNodesAllBuffers() will use the new API.

Yeah, let's do it that way unless anyone has a better idea to suggest.
--
With Regards,
Amit Kapila.

#235Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#231)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Dec 23, 2020 at 10:42 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Wed, 23 Dec 2020 04:22:19 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Amit Kapila <amit.kapila16@gmail.com>

+ /* Get the number of blocks for a relation's fork */
+ block[i][j] = smgrnblocks(smgr_reln[i], j, &cached);
+
+ if (!cached)
+ goto buffer_full_scan;

Why do we need to use goto here? We can simply break from the loop and
then check if (cached && nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD). I think we should try to avoid goto if
possible without much complexity.

That's because two for loops are nested -- breaking there only exits the inner loop. (I thought the same as you at first... And I understand some people have alergy to goto, I think modest use of goto makes the code readable.)

I don't strongly oppose to goto's but in this case the outer loop can
break on the same condition with the inner loop, since cached is true
whenever the inner loop runs to the end. It is needed to initialize
the variable cache with true, instead of false, though.

+1. I think it is better to avoid goto here as it can be done without
introducing any complexity or making code any less readable.

--
With Regards,
Amit Kapila.

#236k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#235)
3 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wed, December 23, 2020 5:57 PM (GMT+9), Amit Kapila wrote:

At Wed, 23 Dec 2020 04:22:19 +0000, "tsunakawa.takay@fujitsu.com"
<tsunakawa.takay@fujitsu.com> wrote in

From: Amit Kapila <amit.kapila16@gmail.com>

+ /* Get the number of blocks for a relation's fork */ block[i][j]
+ = smgrnblocks(smgr_reln[i], j, &cached);
+
+ if (!cached)
+ goto buffer_full_scan;

Why do we need to use goto here? We can simply break from the loop
and then check if (cached && nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD). I think we should try to avoid

goto

if possible without much complexity.

That's because two for loops are nested -- breaking there only exits
the inner loop. (I thought the same as you at first... And I
understand some people have alergy to goto, I think modest use of
goto makes the code readable.)

I don't strongly oppose to goto's but in this case the outer loop can
break on the same condition with the inner loop, since cached is true
whenever the inner loop runs to the end. It is needed to initialize
the variable cache with true, instead of false, though.

+1. I think it is better to avoid goto here as it can be done without
introducing any complexity or making code any less readable.

I also do not mind, so I have removed the goto and followed the advice
of all reviewers. It works fine in the latest attached patch 0003.

Attached herewith are the sets of patches. 0002 & 0003 have the following
changes:

1. I have removed the modifications in smgrnblocks(). So the modifications of
other functions that uses smgrnblocks() in the previous patch versions were
also reverted.
2. Introduced a new API smgrnblocks_cached() instead which returns either
a cached size for the specified fork or an InvalidBlockNumber.
Since InvalidBlockNumber is used, I think it is logical not to use the additional
boolean parameter "cached" in the function as it will be redundant.
Although in 0003, I only used the "cached" as a Boolean variable to do the trick
of not using goto.
This function is called both in DropRelFileNodeBuffers() and DropRelFileNodesAllBuffers().
3. Modified some minor comments from the patch and commit logs.

It compiles. Passes the regression tests too.
Your feedbacks are definitely welcome.

Regards,
Kirk Jamison

Attachments:

v37-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v37-0001-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From b207037a0c7a63255803cbc650097ca3ca1682ad Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v37 1/3] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 072bdd1..1ac6d9c 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -473,7 +473,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

v37-0002-Optimize-DropRelFileNodeBuffers-for-recovery.patchapplication/octet-stream; name=v37-0002-Optimize-DropRelFileNodeBuffers-for-recovery.patchDownload
From 6aee28a5a8fa4c1ec17e34bd503e1893ec05807f Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Dec 2020 08:11:48 +0000
Subject: [PATCH v37 2/3] Optimize DropRelFileNodeBuffers() for recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool can be avoided when the number of blocks
to be truncated in a relation is below a certain threshold. For such cases,
we find the buffers by doing lookups in BufMapping table. This improves
the performance by more than 100 times in many cases for small tables
where the server is configured with a large value of shared buffers
(greater than 20GB).

This optimization helps cases (a) when vacuum or autovacuum truncated off
any of the empty pages at the end of a relation, or (b) when the relation is
truncated in the same transaction in which it was created.

This commit introduces a new API smgrnblocks_cached which returns a cached
value for the number of blocks in a relation fork, and it is currently only
used during recovery.  If we are not in recovery or a relation fork does
not exist, then we return an InvalidBlockNumber.  This helps us to determine
the exact size of relation which is required to apply this optimization. The
exact size is required to ensure that we don't leave any buffer for the
relation being dropped as otherwise the background writer or checkpointer
can lead to a PANIC error while flushing buffers corresponding to files that
don't exist.
---
 src/backend/storage/buffer/bufmgr.c | 144 ++++++++++++++++++++++++++++++++++--
 src/backend/storage/smgr/smgr.c     |  22 +++++-
 src/include/storage/bufmgr.h        |   2 +-
 src/include/storage/smgr.h          |   1 +
 4 files changed, 159 insertions(+), 10 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index c5e8707..f66c641 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,14 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+/*
+ * This is the size (in the number of blocks) above which we scan the
+ * entire buffer pool to remove the buffers for all the pages of relation
+ * being dropped. For the relations with size below this threshold, we find
+ * the buffers by doing lookups in BufMapping table.
+ */
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32) (NBuffers / 256)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +481,10 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FindAndDropRelFileNodeBuffers(RelFileNode rnode,
+										  ForkNumber forkNum,
+										  BlockNumber nForkBlock,
+										  BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2965,19 +2977,19 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend rnode;
+	BlockNumber nForkBlock[MAX_FORKNUM];
+	BlockNumber nBlocksToInvalidate = 0;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2991,6 +3003,64 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * To remove all the pages of the specified relation forks from the buffer
+	 * pool, we need to scan the entire buffer pool but we can optimize it by
+	 * finding the buffers from BufMapping table provided we know the exact
+	 * size of each fork of the relation. The exact size is required to ensure
+	 * that we don't leave any buffer for the relation being dropped as
+	 * otherwise the background writer or checkpointer can lead to a PANIC
+	 * error while flushing buffers corresponding to files that don't exist.
+	 *
+	 * To know the exact size, we rely on the size cached for each fork by us
+	 * during recovery which limits the optimization to recovery and on
+	 * standbys but we can easily extend it once we have shared cache for
+	 * relation size.
+	 *
+	 * In recovery, we cache the value returned by the first lseek(SEEK_END)
+	 * and the future writes keeps the cached value up-to-date. See
+	 * smgrextend. It is possible that the value of the first lseek is smaller
+	 * than the actual number of existing blocks in the file due to buggy
+	 * Linux kernels that might not have accounted for the recent write. But
+	 * that should be fine because there must not be any buffers after that
+	 * file size.
+	 *
+	 * XXX We would make the extra lseek call for the unoptimized paths but
+	 * that is okay because we do it just for the first fork and we anyway
+	 * have to scan the entire buffer pool the cost of which is so high that
+	 * the extra lseek call won't make any visible difference. However, we can
+	 * use InRecovery flag to avoid the additional cost but that doesn't seem
+	 * worth it.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlock[i] = smgrnblocks_cached(smgr_reln, forkNum[i]);
+
+		if (nForkBlock[i] == InvalidBlockNumber)
+		{
+			nBlocksToInvalidate = InvalidBlockNumber;
+			break;
+		}
+
+		/* Get the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlock[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * Enter the optimization if the sizes for all forks are cached
+	 * and the total number of blocks to be invalidated is below the
+	 * full scan threshold.
+	 */
+	if (nBlocksToInvalidate != InvalidBlockNumber &&
+		nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+			FindAndDropRelFileNodeBuffers(rnode.node, forkNum[j],
+										  nForkBlock[j], firstDelBlock[j]);
+		return;
+	}
+
 	for (i = 0; i < NBuffers; i++)
 	{
 		BufferDesc *bufHdr = GetBufferDescriptor(i);
@@ -3134,6 +3204,65 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 }
 
 /* ---------------------------------------------------------------------
+ *		FindAndDropRelFileNodeBuffers
+ *
+ *		This function performs look up in BufMapping table and removes from the
+ *		buffer pool all the pages of the specified relation fork that has block
+ *		number >= firstDelBlock. (In particular, with firstDelBlock = 0, all
+ *		pages are removed.)
+ * --------------------------------------------------------------------
+ */
+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber forkNum,
+							  BlockNumber nForkBlock,
+							  BlockNumber firstDelBlock)
+{
+	BlockNumber curBlock;
+
+	for (curBlock = firstDelBlock; curBlock < nForkBlock; curBlock++)
+	{
+		uint32		bufHash;	/* hash value for tag */
+		BufferTag	bufTag;		/* identity of requested block */
+		LWLock	   *bufPartitionLock;	/* buffer partition lock for it */
+		int			buf_id;
+		BufferDesc *bufHdr;
+		uint32		buf_state;
+
+		/* create a tag so we can lookup the buffer */
+		INIT_BUFFERTAG(bufTag, rnode, forkNum, curBlock);
+
+		/* determine its hash code and partition lock ID */
+		bufHash = BufTableHashCode(&bufTag);
+		bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+		/* Check that it is in the buffer pool. If not, do nothing. */
+		LWLockAcquire(bufPartitionLock, LW_SHARED);
+		buf_id = BufTableLookup(&bufTag, bufHash);
+		LWLockRelease(bufPartitionLock);
+
+		if (buf_id < 0)
+			continue;
+
+		bufHdr = GetBufferDescriptor(buf_id);
+
+		/*
+		 * We need to lock the buffer header and recheck if the buffer is
+		 * still associated with the same block because the buffer could be
+		 * evicted by some other backend loading blocks for a different
+		 * relation after we release lock on the BufMapping table.
+		 */
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+/* ---------------------------------------------------------------------
  *		DropDatabaseBuffers
  *
  *		This function removes all the buffers in the buffer cache for a
@@ -3245,8 +3374,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 1ac6d9c..e5d11ab 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -571,6 +571,26 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 }
 
 /*
+ *	smgrnblocks_cached() -- Get the cached number of blocks in the supplied
+ *							relation.
+ *
+ * It is equivalent to calling smgrnblocks(), but only used in recovery for
+ * now, since there is no shared invalidation mechanism that is implemented
+ * yet for changes in file size.
+ *
+ * The function returns an InvalidBlockNumber when not in recovery and when
+ * the fork size is not cached.
+ */
+BlockNumber
+smgrnblocks_cached(SMgrRelation reln, ForkNumber forknum)
+{
+	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
+		return reln->smgr_cached_nblocks[forknum];
+
+	return InvalidBlockNumber;
+}
+
+/*
  *	smgrtruncate() -- Truncate the given forks of supplied relation to
  *					  each specified numbers of blocks
  *
@@ -589,7 +609,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8f..056f65e 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842..137a15a 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -99,6 +99,7 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
 extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks_cached(SMgrRelation reln, ForkNumber forknum);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v37-0003-Optimize-DropRelFileNodesAllBuffers-in-recovery.patchapplication/octet-stream; name=v37-0003-Optimize-DropRelFileNodesAllBuffers-in-recovery.patchDownload
From f8733bcb4970db3891bec678bb29099a7e13e13a Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Dec 2020 12:36:22 +0000
Subject: [PATCH v37 3/3] Optimize DropRelFileNodesAllBuffers() in recovery.

DropRelFileNodesAllBuffers() is optimized to skip the time-consuming
scan of the whole buffer pool during recovery when the relation is
small enough, or when the number of blocks to be invalidated is below
the full scan threshold. This improves the DropRelationFiles()
performance, for example, when the TRUNCATE command truncated off any
of the empty pages at the end of relation.

We first check if all the relations are candidates for optimization
during recovery, by ensuring that a cached size was returned by
smgrnblocks_cached() for a relation fork.  Similar to
DropRelFileNodeBuffers, we invalidate buffer blocks by locating using
BufTableLookup() when it is certain that we know up to what page of
every fork we possiblly have a buffer.  That can be checked only while
recovery through smgrnblocks_cached(), which returns InvalidBlockNumber
if the fork size is not cached.  Otherwise, we proceed to full scan of
the whole buffer pool if a size of a particular relation fork is not
cached, which can happen if there are no updates to that existing fork
during recovery.
---
 src/backend/storage/buffer/bufmgr.c | 97 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     | 14 +++---
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 96 insertions(+), 17 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index f66c641..b9e51a0 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3112,28 +3112,33 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
+DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 {
-	int			i,
-				n = 0;
+	int			i;
+	int			j;
+	int			n = 0;
+	SMgrRelation	*rels;
+	BlockNumber (*block)[MAX_FORKNUM + 1];
+	BlockNumber	nBlocksToInvalidate = 0;
 	RelFileNode *nodes;
+	bool		cached = true;
 	bool		use_bsearch;
 
 	if (nnodes == 0)
 		return;
 
-	nodes = palloc(sizeof(RelFileNode) * nnodes);	/* non-local relations */
+	rels = palloc(sizeof(SMgrRelation) * nnodes);	/* non-local relations */
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	for (i = 0; i < nnodes; i++)
 	{
-		if (RelFileNodeBackendIsTemp(rnodes[i]))
+		if (RelFileNodeBackendIsTemp(smgr_reln[i]->smgr_rnode))
 		{
-			if (rnodes[i].backend == MyBackendId)
-				DropRelFileNodeAllLocalBuffers(rnodes[i].node);
+			if (smgr_reln[i]->smgr_rnode.backend == MyBackendId)
+				DropRelFileNodeAllLocalBuffers(smgr_reln[i]->smgr_rnode.node);
 		}
 		else
-			nodes[n++] = rnodes[i].node;
+			rels[n++] = smgr_reln[i];
 	}
 
 	/*
@@ -3142,10 +3147,83 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	 */
 	if (n == 0)
 	{
-		pfree(nodes);
+		pfree(rels);
 		return;
 	}
 
+	block = (BlockNumber (*)[MAX_FORKNUM + 1])
+			palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));
+
+	/*
+	 * Check if all the relations are candidates for buffer drop
+	 * optimization.  Otherwise, we proceed to full scan of the
+	 * whole buffer pool if a size of a particular relation fork
+	 * is not cached, which can happen if there are no updates
+	 * to that fork during recovery.
+	 */
+	for (i = 0; i < n && cached; i++)
+	{
+		for (j = 0; j <= MAX_FORKNUM; j++)
+		{
+			/*
+			 * Get the number of blocks for a relation's fork.
+			 * If we don't have a cached size, then it returns
+			 * InvalidBlockNumber.
+			 */
+			block[i][j] = smgrnblocks_cached(rels[i], j);
+
+			/*
+			 * If the fork size is not cached, check whether or not
+			 * the relation fork exists.  Otherwise, we can skip
+			 * the possibility of costly checks especially if there
+			 * are many relations.  If a fork does not exist, we can
+			 * skip it later when dropping the relation buffers.
+			 */
+			if (block[i][j] == InvalidBlockNumber)
+			{
+				if (!smgrexists(rels[i], j))
+					continue;
+				cached = false;
+				break;
+			}
+
+			nBlocksToInvalidate += block[i][j];
+		}
+	}
+
+	/*
+	 * Enter the optimization if the sizes for all relation forks are
+	 * cached and the total number of blocks to be invalidated for all
+	 * relations is below the full scan threshold.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (i = 0; i < n; i++)
+		{
+			/*
+			 * If block to drop is valid, drop the buffers of the fork.
+			 * Zero the firstDelBlock because all buffers will be
+			 * dropped anyway.
+			 */
+			for (j = 0; j <= MAX_FORKNUM; j++)
+			{
+				if (!BlockNumberIsValid(block[i][j]))
+					continue;
+
+				FindAndDropRelFileNodeBuffers(rels[i]->smgr_rnode.node,
+											  j, block[i][j], 0);
+			}
+		}
+		pfree(block);
+		pfree(rels);
+		return;
+	}
+
+	pfree(block);
+	nodes = palloc(sizeof(RelFileNode) * n); /* non-local relations */
+	for (i = 0; i < n; i++)
+		nodes[i] = rels[i]->smgr_rnode.node;
+
 	/*
 	 * For low number of relations to drop just use a simple walk through, to
 	 * save the bsearch overhead. The threshold to use is rather a guess than
@@ -3201,6 +3279,7 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	}
 
 	pfree(nodes);
+	pfree(rels);
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index e5d11ab..7162b18 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -391,7 +391,13 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 		return;
 
 	/*
-	 * create an array which contains all relations to be dropped, and close
+	 * Get rid of any remaining buffers for the relations.  bufmgr will just
+	 * drop them without bothering to write the contents.
+	 */
+	DropRelFileNodesAllBuffers(rels, nrels);
+
+	/*
+	 * Create an array which contains all relations to be dropped, and close
 	 * each relation's forks at the smgr level while at it
 	 */
 	rnodes = palloc(sizeof(RelFileNodeBackend) * nrels);
@@ -408,12 +414,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	}
 
 	/*
-	 * Get rid of any remaining buffers for the relations.  bufmgr will just
-	 * drop them without bothering to write the contents.
-	 */
-	DropRelFileNodesAllBuffers(rnodes, nrels);
-
-	/*
 	 * It'd be nice to tell the stats collector to forget them immediately,
 	 * too. But we can't because we don't know the OIDs.
 	 */
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 056f65e..2e5189b 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -205,7 +205,7 @@ extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels)
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
-extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
+extern void DropRelFileNodesAllBuffers(struct SMgrRelationData **smgr_reln, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
 #define RelationGetNumberOfBlocks(reln) \
-- 
1.8.3.1

#237tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: k.jamison@fujitsu.com (#236)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>
compiles. Passes the regression tests too.

Your feedbacks are definitely welcome.

The code looks correct and has become further compact. Remains ready for committer.

Regards
Takayuki Tsunakawa

#238Tang, Haiying
tanghy.fnst@cn.fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#237)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi Amit, Kirk

One idea could be to remove "nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD" part of check "if (cached &&
nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)" so that it always
use optimized path for the tests. Then use the relation size as
NBuffers/128, NBuffers/256, NBuffers/512 for different values of
shared buffers as 128MB, 1GB, 20GB, 100GB.

I followed your idea to remove check and use different relation size for different shared buffers as 128M,1G,20G,50G(my environment can't support 100G, so I choose 50G).
According to results, all three thresholds can get optimized, even NBuffers/128 when shared_buffers > 128M.
IMHO, I think NBuffers/128 is the maximum relation size we can get optimization in the three thresholds, Please let me know if I made something wrong.

Recovery after vacuum test results as below ' Optimized percentage' and ' Optimization details(unit: second)' shows:
(512),(256),(128): means relation size is NBuffers/512, NBuffers/256, NBuffers/128
%reg: means (patched(512)- master(512))/ master(512)

Optimized percentage:
shared_buffers %reg(512) %reg(256) %reg(128)
-----------------------------------------------------------------
128M 0% -1% -1%
1G -65% -49% -62%
20G -98% -98% -98%
50G -99% -99% -99%

Optimization details(unit: second):
shared_buffers master(512) patched(512) master(256) patched(256) master(128) patched(128)
-----------------------------------------------------------------------------------------------------------------------------
128M 0.108 0.108 0.109 0.108 0.109 0.108
1G 0.310 0.107 0.410 0.208 0.811 0.309
20G 94.493 1.511 188.777 3.014 380.633 6.020
50G 537.978 3.815 867.453 7.524 1559.076 15.541

Test prepare:
Below is test table amount for different shared buffers. Each table size is 8k, so I use table amount = NBuffers/(512 or 256 or 128):
shared_buffers NBuffers NBuffers/512 NBuffers/256 NBuffers/128
-------------------------------------------------------------------------------------------
128M 16384 32 64 128
1G 131072 256 512 1024
20G 2621440 5120 10240 20480
50G 6553600 12800 25600 51200

Besides, I also did single table performance test.
Still, NBuffers/128 is the max relation size which we can get optimization.

Optimized percentage:
shared_buffers %reg(512) %reg(256) %reg(128)
-----------------------------------------------------------------
128M 0% 0% -1%
1G 0% 1% 0%
20G 0% -24% -25%
50G 0% -24% -20%

Optimization details(unit: second):
shared_buffers master(512) patched(512) master(256) patched(256) master(128) patched(128)
-----------------------------------------------------------------------------------------------------------------------------
128M 0.107 0.107 0.108 0.108 0.108 0.107
1G 0.108 0.108 0.107 0.108 0.108 0.108
20G 0.208 0.208 0.409 0.309 0.409 0.308
50G 0.309 0.308 0.408 0.309 0.509 0.408

Any question on my test results is welcome.

Regards,
Tang

#239Amit Kapila
amit.kapila16@gmail.com
In reply to: Tang, Haiying (#238)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Dec 24, 2020 at 2:31 PM Tang, Haiying
<tanghy.fnst@cn.fujitsu.com> wrote:

Hi Amit, Kirk

One idea could be to remove "nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD" part of check "if (cached &&
nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)" so that it always
use optimized path for the tests. Then use the relation size as
NBuffers/128, NBuffers/256, NBuffers/512 for different values of
shared buffers as 128MB, 1GB, 20GB, 100GB.

I followed your idea to remove check and use different relation size for different shared buffers as 128M,1G,20G,50G(my environment can't support 100G, so I choose 50G).
According to results, all three thresholds can get optimized, even NBuffers/128 when shared_buffers > 128M.
IMHO, I think NBuffers/128 is the maximum relation size we can get optimization in the three thresholds, Please let me know if I made something wrong.

But how can we conclude NBuffers/128 is the maximum relation size?
Because the maximum size would be where the performance is worse than
the master, no? I guess we need to try by NBuffers/64, NBuffers/32,
.... till we get the threshold where master performs better.

Recovery after vacuum test results as below ' Optimized percentage' and ' Optimization details(unit: second)' shows:
(512),(256),(128): means relation size is NBuffers/512, NBuffers/256, NBuffers/128
%reg: means (patched(512)- master(512))/ master(512)

Optimized percentage:
shared_buffers %reg(512) %reg(256) %reg(128)
-----------------------------------------------------------------
128M 0% -1% -1%
1G -65% -49% -62%
20G -98% -98% -98%
50G -99% -99% -99%

Optimization details(unit: second):
shared_buffers master(512) patched(512) master(256) patched(256) master(128) patched(128)
-----------------------------------------------------------------------------------------------------------------------------
128M 0.108 0.108 0.109 0.108 0.109 0.108
1G 0.310 0.107 0.410 0.208 0.811 0.309
20G 94.493 1.511 188.777 3.014 380.633 6.020
50G 537.978 3.815 867.453 7.524 1559.076 15.541

I think we should find a better way to display these numbers because
in cases like where master takes 537.978s and patch takes 3.815s, it
is clear that patch has reduced the time by more than 100 times
whereas in your table it shows 99%.

Test prepare:
Below is test table amount for different shared buffers. Each table size is 8k,

Table size should be more than 8k to get all this data because 8k
means just one block. I guess either it is a typo or some other
mistake.

--
With Regards,
Amit Kapila.

#240k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Tang, Haiying (#238)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thu, December 24, 2020 6:02 PM JST, Tang, Haiying wrote:

Hi Amit, Kirk

One idea could be to remove "nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD" part of check "if (cached &&
nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)" so that it

always

use optimized path for the tests. Then use the relation size as
NBuffers/128, NBuffers/256, NBuffers/512 for different values of shared
buffers as 128MB, 1GB, 20GB, 100GB.

I followed your idea to remove check and use different relation size for
different shared buffers as 128M,1G,20G,50G(my environment can't support
100G, so I choose 50G).
According to results, all three thresholds can get optimized, even
NBuffers/128 when shared_buffers > 128M.
IMHO, I think NBuffers/128 is the maximum relation size we can get
optimization in the three thresholds, Please let me know if I made something
wrong.

Hello Tang,
Thank you very much again for testing. Perhaps there is a confusing part in the
presented table where you indicated master(512), master(256), master(128).
Because the master is not supposed to use the BUF_DROP_FULL_SCAN_THRESHOLD
and just execute the existing default full scan of NBuffers.
Or I may have misunderstood something?

Recovery after vacuum test results as below ' Optimized percentage' and '
Optimization details(unit: second)' shows:
(512),(256),(128): means relation size is NBuffers/512, NBuffers/256,
NBuffers/128
%reg: means (patched(512)- master(512))/ master(512)

Optimized percentage:
shared_buffers%reg(512)%reg(256)%reg(128)
-----------------------------------------------------------------
128M0%-1%-1%
1G -65%-49%-62%
20G -98%-98%-98%
50G -99%-99%-99%

Optimization details(unit: second):
shared_buffersmaster(512)patched(512)master(256)patched(256)master(12
8)patched(128)
-------------------------------------------------------------------------------------
----------------------------------------
128M0.1080.1080.1090.1080.1090.108
1G0.310 0.107 0.410 0.208 0.811 0.309
20G 94.493 1.511 188.777 3.014 380.633 6.020
50G537.9783.815867.4537.5241559.07615.541

Test prepare:
Below is test table amount for different shared buffers. Each table size is 8k,
so I use table amount = NBuffers/(512 or 256 or 128):
shared_buffersNBuffersNBuffers/512NBuffers/256NBuffers/128
-------------------------------------------------------------------------------------
------
128M163843264128
1G1310722565121024
20G2621440 51201024020480
50G6553600 128002560051200

Besides, I also did single table performance test.
Still, NBuffers/128 is the max relation size which we can get optimization.

Optimized percentage:
shared_buffers%reg(512)%reg(256)%reg(128)
-----------------------------------------------------------------
128M0%0%-1%
1G 0%1%0%
20G 0%-24%-25%
50G 0%-24%-20%

Optimization details(unit: second):
shared_buffersmaster(512)patched(512)master(256)patched(256)master(12
8)patched(128)
-------------------------------------------------------------------------------------
----------------------------------------
128M0.1070.1070.1080.1080.1080.107
1G0.108 0.108 0.107 0.108 0.108 0.108
20G0.208 0.208 0.409 0.309 0.409 0.308
50G0.309 0.308 0.408 0.309 0.509 0.408

I will also post results from my machine in the next email.
Adding what Amit mentioned that we should also test for NBuffers/64, etc.
until we determine which of the threshold performs worse than master.

Regards,
Kirk Jamison

#241Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#236)
3 attachment(s)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Dec 23, 2020 at 6:27 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

It compiles. Passes the regression tests too.
Your feedbacks are definitely welcome.

Thanks, the patches look good to me now. I have slightly edited the
patches for comments, commit messages, and removed the duplicate
code/check in smgrnblocks. I have changed the order of patches (moved
Assert related patch to last because as mentioned earlier, I am not
sure if we want to commit it.). We might still have to change the scan
threshold value based on your and Tang-San's results.

--
With Regards,
Amit Kapila.

Attachments:

v38-0001-Optimize-DropRelFileNodeBuffers-for-recovery.patchapplication/octet-stream; name=v38-0001-Optimize-DropRelFileNodeBuffers-for-recovery.patchDownload
From a3eaddb8944ae66616e256fe69098d865b6483c4 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Dec 2020 08:11:48 +0000
Subject: [PATCH v38 1/3] Optimize DropRelFileNodeBuffers() for recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool can be avoided when the number of blocks
to be truncated in a relation is below a certain threshold. For such cases,
we find the buffers by doing lookups in BufMapping table. This improves
the performance by more than 100 times in many cases for small tables
where the server is configured with a large value of shared buffers
(greater than 20GB).

This optimization helps cases (a) when vacuum or autovacuum truncated off
any of the empty pages at the end of a relation, or (b) when the relation is
truncated in the same transaction in which it was created.

This commit introduces a new API smgrnblocks_cached which returns a cached
value for the number of blocks in a relation fork. This helps us to determine
the exact size of relation which is required to apply this optimization. The
exact size is required to ensure that we don't leave any buffer for the
relation being dropped as otherwise the background writer or checkpointer
can lead to a PANIC error while flushing buffers corresponding to files that
don't exist.

Author: Kirk Jamison
Reviewed-by: Kyotaro Horiguchi, Takayuki Tsunakawa, and Amit Kapila
Tested-By: Haiying Tang
Discussion: https://postgr.es/m/OSBPR01MB3207DCA7EC725FDD661B3EDAEF660@OSBPR01MB3207.jpnprd01.prod.outlook.com
---
 src/backend/storage/buffer/bufmgr.c | 136 ++++++++++++++++++++++++++--
 src/backend/storage/smgr/smgr.c     |  30 ++++--
 src/include/storage/bufmgr.h        |   2 +-
 src/include/storage/smgr.h          |   1 +
 4 files changed, 154 insertions(+), 15 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index c5e8707151..e3cb3c59f6 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,14 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+/*
+ * This is the size (in the number of blocks) above which we scan the
+ * entire buffer pool to remove the buffers for all the pages of relation
+ * being dropped. For the relations with size below this threshold, we find
+ * the buffers by doing lookups in BufMapping table.
+ */
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32) (NBuffers / 256)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +481,10 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FindAndDropRelFileNodeBuffers(RelFileNode rnode,
+										  ForkNumber forkNum,
+										  BlockNumber nForkBlock,
+										  BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2965,19 +2977,19 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend rnode;
+	BlockNumber nForkBlock[MAX_FORKNUM];
+	BlockNumber nBlocksToInvalidate = 0;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2991,6 +3003,56 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * To remove all the pages of the specified relation forks from the buffer
+	 * pool, we need to scan the entire buffer pool but we can optimize it by
+	 * finding the buffers from BufMapping table provided we know the exact
+	 * size of each fork of the relation. The exact size is required to ensure
+	 * that we don't leave any buffer for the relation being dropped as
+	 * otherwise the background writer or checkpointer can lead to a PANIC
+	 * error while flushing buffers corresponding to files that don't exist.
+	 *
+	 * To know the exact size, we rely on the size cached for each fork by us
+	 * during recovery which limits the optimization to recovery and on
+	 * standbys but we can easily extend it once we have shared cache for
+	 * relation size.
+	 *
+	 * In recovery, we cache the value returned by the first lseek(SEEK_END)
+	 * and the future writes keeps the cached value up-to-date. See
+	 * smgrextend. It is possible that the value of the first lseek is smaller
+	 * than the actual number of existing blocks in the file due to buggy
+	 * Linux kernels that might not have accounted for the recent write. But
+	 * that should be fine because there must not be any buffers after that
+	 * file size.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlock[i] = smgrnblocks_cached(smgr_reln, forkNum[i]);
+
+		if (nForkBlock[i] == InvalidBlockNumber)
+		{
+			nBlocksToInvalidate = InvalidBlockNumber;
+			break;
+		}
+
+		/* calculate the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlock[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * We apply the optimization iff the total number of blocks to invalidate
+	 * is below the BUF_DROP_FULL_SCAN_THRESHOLD.
+	 */
+	if (BlockNumberIsValid(nBlocksToInvalidate) &&
+		nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+			FindAndDropRelFileNodeBuffers(rnode.node, forkNum[j],
+										  nForkBlock[j], firstDelBlock[j]);
+		return;
+	}
+
 	for (i = 0; i < NBuffers; i++)
 	{
 		BufferDesc *bufHdr = GetBufferDescriptor(i);
@@ -3133,6 +3195,65 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	pfree(nodes);
 }
 
+/* ---------------------------------------------------------------------
+ *		FindAndDropRelFileNodeBuffers
+ *
+ *		This function performs look up in BufMapping table and removes from the
+ *		buffer pool all the pages of the specified relation fork that has block
+ *		number >= firstDelBlock. (In particular, with firstDelBlock = 0, all
+ *		pages are removed.)
+ * --------------------------------------------------------------------
+ */
+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber forkNum,
+							  BlockNumber nForkBlock,
+							  BlockNumber firstDelBlock)
+{
+	BlockNumber curBlock;
+
+	for (curBlock = firstDelBlock; curBlock < nForkBlock; curBlock++)
+	{
+		uint32		bufHash;	/* hash value for tag */
+		BufferTag	bufTag;		/* identity of requested block */
+		LWLock	   *bufPartitionLock;	/* buffer partition lock for it */
+		int			buf_id;
+		BufferDesc *bufHdr;
+		uint32		buf_state;
+
+		/* create a tag so we can lookup the buffer */
+		INIT_BUFFERTAG(bufTag, rnode, forkNum, curBlock);
+
+		/* determine its hash code and partition lock ID */
+		bufHash = BufTableHashCode(&bufTag);
+		bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+		/* Check that it is in the buffer pool. If not, do nothing. */
+		LWLockAcquire(bufPartitionLock, LW_SHARED);
+		buf_id = BufTableLookup(&bufTag, bufHash);
+		LWLockRelease(bufPartitionLock);
+
+		if (buf_id < 0)
+			continue;
+
+		bufHdr = GetBufferDescriptor(buf_id);
+
+		/*
+		 * We need to lock the buffer header and recheck if the buffer is
+		 * still associated with the same block because the buffer could be
+		 * evicted by some other backend loading blocks for a different
+		 * relation after we release lock on the BufMapping table.
+		 */
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
 /* ---------------------------------------------------------------------
  *		DropDatabaseBuffers
  *
@@ -3245,8 +3366,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 072bdd118f..cf991ab93d 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -549,6 +549,28 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 {
 	BlockNumber result;
 
+	/* Check and return if we get the cached value for the number of blocks. */
+	result = smgrnblocks_cached(reln, forknum);
+	if (result != InvalidBlockNumber)
+		return result;
+
+	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
+
+	reln->smgr_cached_nblocks[forknum] = result;
+
+	return result;
+}
+
+/*
+ *	smgrnblocks_cached() -- Get the cached number of blocks in the supplied
+ *							relation.
+ *
+ * Returns an InvalidBlockNumber when not in recovery and when the relation
+ * fork size is not cached.
+ */
+BlockNumber
+smgrnblocks_cached(SMgrRelation reln, ForkNumber forknum)
+{
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
 	 * invalidation mechanism for changes in file size.
@@ -556,11 +578,7 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
 		return reln->smgr_cached_nblocks[forknum];
 
-	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
-
-	reln->smgr_cached_nblocks[forknum] = result;
-
-	return result;
+	return InvalidBlockNumber;
 }
 
 /*
@@ -582,7 +600,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ee91b8fa26..056f65efad 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842401..137a15a851 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -99,6 +99,7 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
 extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks_cached(SMgrRelation reln, ForkNumber forknum);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
2.28.0.windows.1

v38-0002-Optimize-DropRelFileNodesAllBuffers-for-recovery.patchapplication/octet-stream; name=v38-0002-Optimize-DropRelFileNodesAllBuffers-for-recovery.patchDownload
From fea5bc48357b7ce78ed3f77c750b682e67c748c7 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Dec 2020 12:36:22 +0000
Subject: [PATCH v38 2/3] Optimize DropRelFileNodesAllBuffers() for recovery.

Similar to commit <>, this patch optimizes DropRelFileNodesAllBuffers() by
avoiding the complete buffer pool scan and instead find the buffers to be
invalidated by doing lookups in the BufMapping table.

This optimization helps operations where the relation files need to be
removed like Truncate, Drop, Abort of Create Table, etc.

Author: Kirk Jamison
Reviewed-by: Kyotaro Horiguchi, Takayuki Tsunakawa, and Amit Kapila
Tested-By: Haiying Tang
Discussion: https://postgr.es/m/OSBPR01MB3207DCA7EC725FDD661B3EDAEF660@OSBPR01MB3207.jpnprd01.prod.outlook.com
---
 src/backend/storage/buffer/bufmgr.c | 86 ++++++++++++++++++++++++++---
 src/backend/storage/smgr/smgr.c     | 12 ++--
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 84 insertions(+), 16 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index e3cb3c59f6..881aada127 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3104,28 +3104,33 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
+DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 {
-	int			i,
-				n = 0;
+	int			i;
+	int			j;
+	int			n = 0;
+	SMgrRelation	*rels;
+	BlockNumber (*block)[MAX_FORKNUM + 1];
+	BlockNumber	nBlocksToInvalidate = 0;
 	RelFileNode *nodes;
+	bool		cached = true;
 	bool		use_bsearch;
 
 	if (nnodes == 0)
 		return;
 
-	nodes = palloc(sizeof(RelFileNode) * nnodes);	/* non-local relations */
+	rels = palloc(sizeof(SMgrRelation) * nnodes);	/* non-local relations */
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	for (i = 0; i < nnodes; i++)
 	{
-		if (RelFileNodeBackendIsTemp(rnodes[i]))
+		if (RelFileNodeBackendIsTemp(smgr_reln[i]->smgr_rnode))
 		{
-			if (rnodes[i].backend == MyBackendId)
-				DropRelFileNodeAllLocalBuffers(rnodes[i].node);
+			if (smgr_reln[i]->smgr_rnode.backend == MyBackendId)
+				DropRelFileNodeAllLocalBuffers(smgr_reln[i]->smgr_rnode.node);
 		}
 		else
-			nodes[n++] = rnodes[i].node;
+			rels[n++] = smgr_reln[i];
 	}
 
 	/*
@@ -3134,10 +3139,72 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	 */
 	if (n == 0)
 	{
-		pfree(nodes);
+		pfree(rels);
+		return;
+	}
+
+	/*
+	 * This is used to remember the number of blocks for all the relations
+	 * forks.
+	 */
+	block = (BlockNumber (*)[MAX_FORKNUM + 1])
+			palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));
+
+	/*
+	 * We can avoid scanning the entire buffer pool if we know the exact size
+	 * of each of the given relation forks. See DropRelFileNodeBuffers.
+	 */
+	for (i = 0; i < n && cached; i++)
+	{
+		for (j = 0; j <= MAX_FORKNUM; j++)
+		{
+			/* Get the number of blocks for a relation's fork. */
+			block[i][j] = smgrnblocks_cached(rels[i], j);
+
+			/* We need to only consider the relation forks that exists. */
+			if (block[i][j] == InvalidBlockNumber)
+			{
+				if (!smgrexists(rels[i], j))
+					continue;
+				cached = false;
+				break;
+			}
+
+			/* calculate the total number of blocks to be invalidated */
+			nBlocksToInvalidate += block[i][j];
+		}
+	}
+
+	/*
+	 * We apply the optimization iff the total number of blocks to invalidate
+	 * is below the BUF_DROP_FULL_SCAN_THRESHOLD.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (i = 0; i < n; i++)
+		{
+			for (j = 0; j <= MAX_FORKNUM; j++)
+			{
+				/* ignore relation forks that doesn't exist */
+				if (!BlockNumberIsValid(block[i][j]))
+					continue;
+
+				/* drop all the buffers for a particular relation fork */
+				FindAndDropRelFileNodeBuffers(rels[i]->smgr_rnode.node,
+											  j, block[i][j], 0);
+			}
+		}
+
+		pfree(block);
+		pfree(rels);
 		return;
 	}
 
+	pfree(block);
+	nodes = palloc(sizeof(RelFileNode) * n); /* non-local relations */
+	for (i = 0; i < n; i++)
+		nodes[i] = rels[i]->smgr_rnode.node;
+
 	/*
 	 * For low number of relations to drop just use a simple walk through, to
 	 * save the bsearch overhead. The threshold to use is rather a guess than
@@ -3193,6 +3260,7 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	}
 
 	pfree(nodes);
+	pfree(rels);
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index cf991ab93d..26478d7185 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -390,6 +390,12 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	if (nrels == 0)
 		return;
 
+	/*
+	 * Get rid of any remaining buffers for the relations.  bufmgr will just
+	 * drop them without bothering to write the contents.
+	 */
+	DropRelFileNodesAllBuffers(rels, nrels);
+
 	/*
 	 * create an array which contains all relations to be dropped, and close
 	 * each relation's forks at the smgr level while at it
@@ -407,12 +413,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 			smgrsw[which].smgr_close(rels[i], forknum);
 	}
 
-	/*
-	 * Get rid of any remaining buffers for the relations.  bufmgr will just
-	 * drop them without bothering to write the contents.
-	 */
-	DropRelFileNodesAllBuffers(rnodes, nrels);
-
 	/*
 	 * It'd be nice to tell the stats collector to forget them immediately,
 	 * too. But we can't because we don't know the OIDs.
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 056f65efad..2e5189bb8c 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -205,7 +205,7 @@ extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels)
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
-extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
+extern void DropRelFileNodesAllBuffers(struct SMgrRelationData **smgr_reln, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
 #define RelationGetNumberOfBlocks(reln) \
-- 
2.28.0.windows.1

v38-0003-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v38-0003-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From 044344495adea5cd614af961345e6f1d7278971b Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 23 Sep 2020 07:15:30 +0000
Subject: [PATCH v38 3/3] Prevent invalidating blocks in smgrextend() during
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 26478d7185..717bff76bb 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -473,7 +473,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
2.28.0.windows.1

#242Tang, Haiying
tanghy.fnst@cn.fujitsu.com
In reply to: k.jamison@fujitsu.com (#240)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi Kirk,

Perhaps there is a confusing part in the presented table where you indicated master(512), master(256), master(128).
Because the master is not supposed to use the BUF_DROP_FULL_SCAN_THRESHOLD and just execute the existing default full scan of NBuffers.
Or I may have misunderstood something?

Sorry for your confusion, I didn't make it clear. I didn't use BUF_DROP_FULL_SCAN_THRESHOLD for master.
Master(512) means the test table amount in master is same with patched(512), so does master(256) and master(128).
I meant to mark 512/256/128 to distinguish results in master for the three threshold(applied in patches) .

Regards
Tang

#243Tang, Haiying
tanghy.fnst@cn.fujitsu.com
In reply to: Amit Kapila (#239)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi Amit,

But how can we conclude NBuffers/128 is the maximum relation size?
Because the maximum size would be where the performance is worse than
the master, no? I guess we need to try by NBuffers/64, NBuffers/32,
.... till we get the threshold where master performs better.

You are right, we should keep on testing until no optimization.

I think we should find a better way to display these numbers because in
cases like where master takes 537.978s and patch takes 3.815s

Yeah, I think we can change the %reg formula from (patched- master)/ master to (patched- master)/ patched.

Table size should be more than 8k to get all this data because 8k means
just one block. I guess either it is a typo or some other mistake.

8k here is the relation size, not data size.
For example, when I tested recovery performance of 400M relation size, I used 51200 tables(8k per table).
Please let me know if you think this is not appropriate.

Regards
Tang

-----Original Message-----
From: Amit Kapila <amit.kapila16@gmail.com>
Sent: Thursday, December 24, 2020 9:11 PM
To: Tang, Haiying/唐 海英 <tanghy.fnst@cn.fujitsu.com>
Cc: Tsunakawa, Takayuki/綱川 貴之 <tsunakawa.takay@fujitsu.com>; Jamison, Kirk/ジャミソン カーク <k.jamison@fujitsu.com>; Kyotaro Horiguchi <horikyota.ntt@gmail.com>; Andres Freund <andres@anarazel.de>; Tom Lane <tgl@sss.pgh.pa.us>; Thomas Munro <thomas.munro@gmail.com>; Robert Haas <robertmhaas@gmail.com>; Tomas Vondra <tomas.vondra@2ndquadrant.com>; pgsql-hackers <pgsql-hackers@postgresql.org>
Subject: Re: [Patch] Optimize dropping of relation buffers using dlist

On Thu, Dec 24, 2020 at 2:31 PM Tang, Haiying <tanghy.fnst@cn.fujitsu.com> wrote:

Hi Amit, Kirk

One idea could be to remove "nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD" part of check "if (cached &&
nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)" so that it
always use optimized path for the tests. Then use the relation size
as NBuffers/128, NBuffers/256, NBuffers/512 for different values of
shared buffers as 128MB, 1GB, 20GB, 100GB.

I followed your idea to remove check and use different relation size for different shared buffers as 128M,1G,20G,50G(my environment can't support 100G, so I choose 50G).
According to results, all three thresholds can get optimized, even NBuffers/128 when shared_buffers > 128M.
IMHO, I think NBuffers/128 is the maximum relation size we can get optimization in the three thresholds, Please let me know if I made something wrong.

But how can we conclude NBuffers/128 is the maximum relation size?
Because the maximum size would be where the performance is worse than the master, no? I guess we need to try by NBuffers/64, NBuffers/32, .... till we get the threshold where master performs better.

Recovery after vacuum test results as below ' Optimized percentage' and ' Optimization details(unit: second)' shows:
(512),(256),(128): means relation size is NBuffers/512, NBuffers/256,
NBuffers/128
%reg: means (patched(512)- master(512))/ master(512)

Optimized percentage:
shared_buffers %reg(512) %reg(256) %reg(128)
-----------------------------------------------------------------
128M 0% -1% -1%
1G -65% -49% -62%
20G -98% -98% -98%
50G -99% -99% -99%

Optimization details(unit: second):
shared_buffers master(512) patched(512) master(256) patched(256) master(128) patched(128)
-----------------------------------------------------------------------------------------------------------------------------
128M 0.108 0.108 0.109 0.108 0.109 0.108
1G 0.310 0.107 0.410 0.208 0.811 0.309
20G 94.493 1.511 188.777 3.014 380.633 6.020
50G 537.978 3.815 867.453 7.524 1559.076 15.541

I think we should find a better way to display these numbers because in cases like where master takes 537.978s and patch takes 3.815s, it is clear that patch has reduced the time by more than 100 times whereas in your table it shows 99%.

Test prepare:
Below is test table amount for different shared buffers. Each table
size is 8k,

Table size should be more than 8k to get all this data because 8k means just one block. I guess either it is a typo or some other mistake.

--
With Regards,
Amit Kapila.

#244Amit Kapila
amit.kapila16@gmail.com
In reply to: Tang, Haiying (#243)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Dec 25, 2020 at 9:28 AM Tang, Haiying
<tanghy.fnst@cn.fujitsu.com> wrote:

Hi Amit,

But how can we conclude NBuffers/128 is the maximum relation size?
Because the maximum size would be where the performance is worse than
the master, no? I guess we need to try by NBuffers/64, NBuffers/32,
.... till we get the threshold where master performs better.

You are right, we should keep on testing until no optimization.

I think we should find a better way to display these numbers because in
cases like where master takes 537.978s and patch takes 3.815s

Yeah, I think we can change the %reg formula from (patched- master)/ master to (patched- master)/ patched.

Table size should be more than 8k to get all this data because 8k means
just one block. I guess either it is a typo or some other mistake.

8k here is the relation size, not data size.
For example, when I tested recovery performance of 400M relation size, I used 51200 tables(8k per table).
Please let me know if you think this is not appropriate.

I think one table with a varying amount of data is sufficient for the
vacuum test. I think with more number of tables there is a greater
chance of variation. We have previously used multiple tables in one of
the tests because of the Truncate operation (which uses
DropRelFileNodesAllBuffers that takes multiple relations as input) and
that is not true for Vacuum operation which I suppose you are testing
here.

--
With Regards,
Amit Kapila.

#245Tang, Haiying
tanghy.fnst@cn.fujitsu.com
In reply to: Amit Kapila (#244)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi Amit,

I think one table with a varying amount of data is sufficient for the vacuum test.
I think with more number of tables there is a greater chance of variation.
We have previously used multiple tables in one of the tests because of the
Truncate operation (which uses DropRelFileNodesAllBuffers that takes multiple relations as input)
and that is not true for Vacuum operation which I suppose you are testing here.

Thanks for your advice and kindly explanation. I'll continue the threshold test with one single table.

Regards,
Tang

#246Tang, Haiying
tanghy.fnst@cn.fujitsu.com
In reply to: Tang, Haiying (#245)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi Amit,

I think one table with a varying amount of data is sufficient for the vacuum test.
I think with more number of tables there is a greater chance of variation.
We have previously used multiple tables in one of the tests because of
the Truncate operation (which uses DropRelFileNodesAllBuffers that
takes multiple relations as input) and that is not true for Vacuum operation which I suppose you are testing here.

I retested performance on single table for several times, the table size is varying with the BUF_DROP_FULL_SCAN_THRESHOLD for different shared buffers.
When shared_buffers is below 20G, there were no significant changes between master(HEAD) and patched.
And according to the results compared between 20G and 100G, we can get optimization up to NBuffers/128, but there is no benefit from NBuffers/256.
I've tested many times, most times the same results came out, I don't know why. But If I used 5 tables(each table size is set as BUF_DROP_FULL_SCAN_THRESHOLD), then we can get benefit from it(NBuffers/256).

Here is my test results for single table. If you have any question or suggestion, kindly let me know.

%reg= (patched- master(HEAD))/ patched
Optimized percentage:
shared_buffers %reg(NBuffers/512) %reg(NBuffers/256) %reg(NBuffers/128) %reg(NBuffers/64) %reg(NBuffers/32) %reg(NBuffers/16) %reg(NBuffers/8) %reg(NBuffers/4)
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
128M 0% 0% -1% 0% 1% 0% 0% 0%
1G -1% 0% -1% 0% 0% 0% 0% 0%
20G 0% 0% -33% 0% 0% -13% 0% 0%
100G -32% 0% -49% 0% 10% 30% 0% 6%

Result details(unit: second):
patched (sec)
shared_buffers NBuffers/512 NBuffers/256 NBuffers/128 NBuffers/64 NBuffers/32 NBuffers/16 NBuffers/8 NBuffers/4
128M 0.107 0.107 0.107 0.107 0.108 0.107 0.108 0.208
1G 0.107 0.107 0.107 0.108 0.208 0.208 0.308 0.409
20G 0.208 0.308 0.308 0.409 0.609 0.808 1.511 2.713
100G 0.309 0.408 0.609 1.010 2.011 5.017 6.620 13.931

master(HEAD) (sec)
shared_buffers NBuffers/512 NBuffers/256 NBuffers/128 NBuffers/64 NBuffers/32 NBuffers/16 NBuffers/8 NBuffers/4
128M 0.107 0.107 0.108 0.107 0.107 0.107 0.108 0.208
1G 0.108 0.107 0.108 0.108 0.208 0.207 0.308 0.409
20G 0.208 0.309 0.409 0.409 0.609 0.910 1.511 2.712
100G 0.408 0.408 0.909 1.010 1.811 3.515 6.619 13.032

Regards
Tang

#247Tang, Haiying
tanghy.fnst@cn.fujitsu.com
In reply to: Tang, Haiying (#246)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi Amit,

In last mail(/messages/by-id/66851e198f6b41eda59e6257182564b6@G08CNEXMBPEKD05.g08.fujitsu.local),
I've sent you the performance test results(run only 1 time) on single table. Here is my the retested results(average by 15 times) which I think is more accurate.

In terms of 20G and 100G, the optimization on 100G is linear, but 20G is nonlinear(also include test results on shared buffers of 50G/60G), so it's a little difficult to decide the threshold from the two for me.
If just consider 100G, I think NBuffers/32 is the optimized max relation size. But I don't know how to judge for 20G. If you have any suggestion, kindly let me know.

#%reg 128M 1G 20G 100G
---------------------------------------------------------------
%reg(NBuffers/512) 0% -1% -5% -26%
%reg(NBuffers/256) 0% 0% 5% -20%
%reg(NBuffers/128) -1% -1% -10% -16%
%reg(NBuffers/64) -1% 0% 0% -8%
%reg(NBuffers/32) 0% 0% -2% -4%
%reg(NBuffers/16) 0% 0% -6% 4%
%reg(NBuffers/8) 1% 0% 2% -2%
%reg(NBuffers/4) 0% 0% 2% 2%

Optimization details(unit: second):
patched (sec)
shared_buffers NBuffers/512 NBuffers/256 NBuffers/128 NBuffers/64 NBuffers/32 NBuffers/16 NBuffers/8 NBuffers/4
----------------------------------------------------------------------------------------------------------------------------------------------------------
128M 0.107 0.107 0.107 0.107 0.107 0.107 0.108 0.208
1G 0.107 0.108 0.107 0.108 0.208 0.208 0.308 0.409
20G 0.199 0.299 0.317 0.408 0.591 0.900 1.561 2.866
100G 0.318 0.381 0.645 0.992 1.913 3.640 6.615 13.389

master(HEAD) (sec)
shared_buffers NBuffers/512 NBuffers/256 NBuffers/128 NBuffers/64 NBuffers/32 NBuffers/16 NBuffers/8 NBuffers/4
----------------------------------------------------------------------------------------------------------------------------------------------------------
128M 0.107 0.107 0.108 0.108 0.107 0.107 0.107 0.208
1G 0.108 0.108 0.108 0.108 0.208 0.207 0.308 0.409
20G 0.208 0.283 0.350 0.408 0.601 0.955 1.529 2.806
100G 0.400 0.459 0.751 1.068 1.984 3.506 6.735 13.101

Regards
Tang

#248Amit Kapila
amit.kapila16@gmail.com
In reply to: Tang, Haiying (#247)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Dec 30, 2020 at 11:28 AM Tang, Haiying
<tanghy.fnst@cn.fujitsu.com> wrote:

Hi Amit,

In last mail(/messages/by-id/66851e198f6b41eda59e6257182564b6@G08CNEXMBPEKD05.g08.fujitsu.local),
I've sent you the performance test results(run only 1 time) on single table. Here is my the retested results(average by 15 times) which I think is more accurate.

In terms of 20G and 100G, the optimization on 100G is linear, but 20G is nonlinear(also include test results on shared buffers of 50G/60G), so it's a little difficult to decide the threshold from the two for me.
If just consider 100G, I think NBuffers/32 is the optimized max relation size. But I don't know how to judge for 20G. If you have any suggestion, kindly let me know.

Considering these results NBuffers/64 seems a good threshold as beyond
that there is no big advantage. BTW, it is not clear why the advantage
for single table is not as big as multiple tables with the Truncate
command. Can you share your exact test steps for any one of the tests?
Also, did you change autovacumm = off for these tests, if not then the
results might not be reliable because before you run the test via
Vacuum command autovacuum would have done that work?

--
With Regards,
Amit Kapila.

#249k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#248)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wednesday, December 30, 2020 8:58 PM, Amit Kapila wrote:

On Wed, Dec 30, 2020 at 11:28 AM Tang, Haiying
<tanghy.fnst@cn.fujitsu.com> wrote:

Hi Amit,

In last

mail(/messages/by-id/66851e198f6b41eda59e625718
2

564b6%40G08CNEXMBPEKD05.g08.fujitsu.local),
I've sent you the performance test results(run only 1 time) on single table.

Here is my the retested results(average by 15 times) which I think is more
accurate.

In terms of 20G and 100G, the optimization on 100G is linear, but 20G is

nonlinear(also include test results on shared buffers of 50G/60G), so it's a
little difficult to decide the threshold from the two for me.

If just consider 100G, I think NBuffers/32 is the optimized max relation size.

But I don't know how to judge for 20G. If you have any suggestion, kindly let
me know.

Considering these results NBuffers/64 seems a good threshold as beyond
that there is no big advantage. BTW, it is not clear why the advantage for
single table is not as big as multiple tables with the Truncate command. Can
you share your exact test steps for any one of the tests?
Also, did you change autovacumm = off for these tests, if not then the results
might not be reliable because before you run the test via Vacuum command
autovacuum would have done that work?

Happy new year. The V38 LGTM.
Apologies for a bit of delay on posting the test results, but since it's the
start of commitfest, here it goes and the results were interesting.

I executed a VACUUM test using the same approach that Tsunakawa-san did in [1]/messages/by-id/TYAPR01MB2990C4EFE63F066F83D2A603FEE70@TYAPR01MB2990.jpnprd01.prod.outlook.com,
but only this time, the total time that DropRelFileNodeBuffers() took.
I used only a single relation, tried with various sizes using the values of threshold:
NBuffers/512..NBuffers/1, as advised by Amit.

Example of relation sizes for NBuffers/512.
100GB shared_buffers: 200 MB
20GB shared_buffers: 40 MB
1GB shared_buffers: 2 MB
128MB shared_buffers: 0.25 MB

The regression, which means the patch performs worse than master, only happens
for relation size NBuffers/2 and below for all shared_buffers. The fastest
performance on a single relation was using the relation size NBuffers/512.

[VACUUM Recovery Performance on Single Relation]
Legend: P_XXX (Patch, NBuffers/XXX relation size),
M_XXX (Master, Nbuffers/XXX relation size)
Unit: seconds

| Rel Size | 100 GB s_b | 20 GB s_b | 1 GB s_b | 128 MB s_b |
|----------|------------|------------|------------|------------|
| P_512 | 0.012594 | 0.001989 | 0.000081 | 0.000012 |
| M_512 | 0.208757 | 0.046212 | 0.002013 | 0.000295 |
| P_256 | 0.026311 | 0.004416 | 0.000129 | 0.000021 |
| M_256 | 0.241017 | 0.047234 | 0.002363 | 0.000298 |
| P_128 | 0.044684 | 0.009784 | 0.000290 | 0.000042 |
| M_128 | 0.253588 | 0.047952 | 0.002454 | 0.000319 |
| P_64 | 0.065806 | 0.017444 | 0.000521 | 0.000075 |
| M_64 | 0.268311 | 0.050361 | 0.002730 | 0.000339 |
| P_32 | 0.121441 | 0.033431 | 0.001646 | 0.000112 |
| M_32 | 0.285254 | 0.061486 | 0.003640 | 0.000364 |
| P_16 | 0.255503 | 0.065492 | 0.001663 | 0.000144 |
| M_16 | 0.377013 | 0.081613 | 0.003731 | 0.000372 |
| P_8 | 0.560616 | 0.109509 | 0.005954 | 0.000465 |
| M_8 | 0.692596 | 0.112178 | 0.006667 | 0.000553 |
| P_4 | 1.109437 | 0.162924 | 0.011229 | 0.000861 |
| M_4 | 1.162125 | 0.178764 | 0.011635 | 0.000935 |
| P_2 | 2.202231 | 0.317832 | 0.020783 | 0.002646 |
| M_2 | 1.583959 | 0.306269 | 0.015705 | 0.002021 |
| P_1 | 3.080032 | 0.632747 | 0.032183 | 0.002660 |
| M_1 | 2.705485 | 0.543970 | 0.030658 | 0.001941 |

%reg = (Patched/Master)/Patched

| %reg_relsize | 100 GB s_b | 20 GB s_b | 1 GB s_b | 128 MB s_b |
|--------------|------------|------------|------------|------------|
| %reg_512 | -1557.587% | -2223.006% | -2385.185% | -2354.167% |
| %reg_256 | -816.041% | -969.691% | -1731.783% | -1319.048% |
| %reg_128 | -467.514% | -390.123% | -747.008% | -658.333% |
| %reg_64 | -307.727% | -188.704% | -423.992% | -352.000% |
| %reg_32 | -134.891% | -83.920% | -121.097% | -225.970% |
| %reg_16 | -47.557% | -24.614% | -124.279% | -157.390% |
| %reg_8 | -23.542% | -2.437% | -11.967% | -19.010% |
| %reg_4 | -4.749% | -9.722% | -3.608% | -8.595% |
| %reg_2 | 28.075% | 3.638% | 24.436% | 23.615% |
| %reg_1 | 12.160% | 14.030% | 4.739% | 27.010% |

Since our goal is to get the approximate threshold where the cost for
finding to be invalidated buffers gets higher in optimized path than
the traditional path:
A. Traditional Path
1. For each shared_buffers, compare the relfilenode.
2. LockBufHdr()
3. Compare block number, InvalidateBuffers() if it's the target.
B. Optimized Path
1. For each block in rleation, LWLockAcquire(), BufTableLookup(),
and LWLockRelease().
2-3. Same as traditional path.

So we have to get the difference in #1, where the number of buffers
and the check for each number of to be invalidated buffers differ.
The cost of optimized path will get higher than the traditional path
at some threshold.

NBuffers * traditional_cost_for_each_buf_check <
InvalidatedBuffers * optimized_cost_for_each_buf_check

So what we want to know as the threshold value is the InvalidatedBuffers.
NBuffers * traditional / optimized < InvalidatedBuffers.

Example for 100GB shared_buffers for rel_size NBuffers/512:
100000(MB) * 0.208757 (s) / 0.012594 (s) = 1,657,587 MB,
which is still above the value of 100,000 MB.

| s_b | 100000 | 20000 | 1000 | 128 |
|--------------|-----------|---------|--------|-------|
| NBuffers/512 | 1,657,587 | 464,601 | 24,852 | 3,141 |
| NBuffers/256 | 916,041 | 213,938 | 18,318 | 1,816 |
| NBuffers/128 | 567,514 | 98,025 | 8,470 | 971 |
| NBuffers/64 | 407,727 | 57,741 | 5,240 | 579 |
| NBuffers/32 | 234,891 | 36,784 | 2,211 | 417 |
| NBuffers/16 | 147,557 | 24,923 | 2,243 | 329 |
| NBuffers/8 | 123,542 | 20,487 | 1,120 | 152 |
| NBuffers/4 | 104,749 | 21,944 | 1,036 | 139 |
| NBuffers/2 | 71,925 | 19,272 | 756 | 98 |
| NBuffers/1 | 87,840 | 17,194 | 953 | 93 |

Although the above table shows that NBuffers/2 would be the
threshold, I know that the cost would vary depending on the machine
specs. I think I can suggest the threshold and pick one from among
NBuffers/2, NBuffers/4 or NBuffers/8, because their values are closer
to the InvalidatedBuffers.

[postgesql.conf]
shared_buffers = 100GB #20GB,1GB,128MB
autovacuum = off
full_page_writes = off
checkpoint_timeout = 30min
max_locks_per_transaction = 10000

[Machine Specs Used]
Intel(R) Xeon(R) CPU E5-2637 v4 @ 3.50GHz
8 CPUs, 256GB Memory
XFS, RHEL7.2

Kindly let me know if you have comments regarding the results.

Regards,
Kirk Jamison

[1]: /messages/by-id/TYAPR01MB2990C4EFE63F066F83D2A603FEE70@TYAPR01MB2990.jpnprd01.prod.outlook.com

#250Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#249)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Sat, Jan 2, 2021 at 7:47 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Happy new year. The V38 LGTM.
Apologies for a bit of delay on posting the test results, but since it's the
start of commitfest, here it goes and the results were interesting.

I executed a VACUUM test using the same approach that Tsunakawa-san did in [1],
but only this time, the total time that DropRelFileNodeBuffers() took.

Please specify the exact steps like did you deleted all the rows from
a table or some of it or none before performing Vacuum? How did you
measure this time, did you removed the cached check? It would be
better if you share the scripts and or the exact steps so that the
same can be used by others to reproduce.

I used only a single relation, tried with various sizes using the values of threshold:
NBuffers/512..NBuffers/1, as advised by Amit.

Example of relation sizes for NBuffers/512.
100GB shared_buffers: 200 MB
20GB shared_buffers: 40 MB
1GB shared_buffers: 2 MB
128MB shared_buffers: 0.25 MB

..

Although the above table shows that NBuffers/2 would be the
threshold, I know that the cost would vary depending on the machine
specs. I think I can suggest the threshold and pick one from among
NBuffers/2, NBuffers/4 or NBuffers/8, because their values are closer
to the InvalidatedBuffers.

Hmm, in the tests done by Tang, the results indicate that in some
cases the patched version is slower at even NBuffers/32, so not sure
if we can go to values shown by you unless she is doing something
wrong. I think the difference in results could be because both of you
are using different techniques to measure the timings, so it might be
better if both of you can share scripts or exact steps used to measure
the time and other can use the same technique and see if we are
getting consistent results.

--
With Regards,
Amit Kapila.

#251Tang, Haiying
tanghy.fnst@cn.fujitsu.com
In reply to: Amit Kapila (#250)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi Amit,

Sorry for my late reply. Here are my answers for your earlier questions.

BTW, it is not clear why the advantage for single table is not as big as multiple tables with the Truncate command

I guess it's the amount of table blocks caused this difference. For single table I tested the amount of block is threshold.
For multiple tables I test the amount of block is a value(like: one or dozens or hundreds) which far below threshold.
The closer table blocks to the threshold, the less advantage raised.

I tested below 3 situations of 50 tables when shared buffers=20G / 100G.
1. For multiple tables which had one or dozens or hundreds blocks(far below threshold) per table, we got significant improve, like [1]. 247 blocks per table s_b master patched %reg((patched-master)/patched) ---------------------------------------------------- 20GB 1.109 0.108 -927% 100GB 3.113 0.108 -2782%.
2. For multiple tables which has half threshold blocks per table, advantage become less, like [2]. NBuffers/256/2 blocks per table s_b master patched %reg ---------------------------------------------------- 20GB 2.012 1.210 -66% 100GB 10.226 6.4 -60%.
3. For multiple tables which has threshold blocks per table, advantage become more less, like [3]. NBuffers/256 blocks per table s_b master patched %reg ---------------------------------------------------- 20GB 3.868 2.412 -60% 100GB 14.977 10.591 -41%.

[1]: . 247 blocks per table s_b master patched %reg((patched-master)/patched) ---------------------------------------------------- 20GB 1.109 0.108 -927% 100GB 3.113 0.108 -2782%
s_b master patched %reg((patched-master)/patched)
----------------------------------------------------
20GB 1.109 0.108 -927%
100GB 3.113 0.108 -2782%

[2]: . NBuffers/256/2 blocks per table s_b master patched %reg ---------------------------------------------------- 20GB 2.012 1.210 -66% 100GB 10.226 6.4 -60%
s_b master patched %reg
----------------------------------------------------
20GB 2.012 1.210 -66%
100GB 10.226 6.4 -60%

[3]: . NBuffers/256 blocks per table s_b master patched %reg ---------------------------------------------------- 20GB 3.868 2.412 -60% 100GB 14.977 10.591 -41%
s_b master patched %reg
----------------------------------------------------
20GB 3.868 2.412 -60%
100GB 14.977 10.591 -41%

Can you share your exact test steps for any one of the tests? Also, did you change autovacumm = off for these tests?

Yes, I configured streaming replication environment as Kirk did before.
autovacumm = off.
full_page_writes = off.
checkpoint_timeout = 30min

Test steps:
e.g. shared_buffers=20G, NBuffers/512, table blocks= 20*1024*1024/8/512=5120 . table size(kB)= 20*1024*1024/512=40960kB
1. (Master) create table test(id int, v_ch varchar, v_ch1 varchar);
2. (Master) insert about 40MB data to table.
3. (Master) delete from table (all rows of table)
4. (Standby) To test with failover, pause the WAL replay on standby server.
SELECT pg_wal_replay_pause();
5. (Master) VACUUM;
6. (Master) Stop primary server. pg_ctl stop -D $PGDATA -w 7. (Standby) Resume wal replay and promote standby. (get the recovery time from this step)

Regards
Tang

#252k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#250)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Sunday, January 3, 2021 10:35 PM (JST), Amit Kapila wrote:

On Sat, Jan 2, 2021 at 7:47 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

Happy new year. The V38 LGTM.
Apologies for a bit of delay on posting the test results, but since
it's the start of commitfest, here it goes and the results were interesting.

I executed a VACUUM test using the same approach that Tsunakawa-san
did in [1], but only this time, the total time that DropRelFileNodeBuffers()

took.

Please specify the exact steps like did you deleted all the rows from a table or
some of it or none before performing Vacuum? How did you measure this
time, did you removed the cached check? It would be better if you share the
scripts and or the exact steps so that the same can be used by others to
reproduce.

Basically, I used the TimestampDifference function in DropRelFileNodeBuffers().
I also executed DELETE before VACUUM.
I also removed nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD
And used the threshold as the relation size.

Hmm, in the tests done by Tang, the results indicate that in some cases the
patched version is slower at even NBuffers/32, so not sure if we can go to
values shown by you unless she is doing something wrong. I think the
difference in results could be because both of you are using different
techniques to measure the timings, so it might be better if both of you can
share scripts or exact steps used to measure the time and other can use the
same technique and see if we are getting consistent results.

Right, since we want consistent results, please disregard the approach that I did.
I will resume the test similar to Tang, because she also executed the original failover
test which I have been doing before.
To avoid confusion and to check if the results from mine and Tang are consistent,
I also did the recovery/failover test for VACUUM on single relation, which I will send
in a separate email after this.

Regards,
Kirk Jamison

#253k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: k.jamison@fujitsu.com (#252)
6 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wed, January 6, 2021 7:04 PM (JST), I wrote:

I will resume the test similar to Tang, because she also executed the original
failover test which I have been doing before.
To avoid confusion and to check if the results from mine and Tang are
consistent, I also did the recovery/failover test for VACUUM on single relation,
which I will send in a separate email after this.

A. Test to find the right THRESHOLD

So below are the procedures and results of the VACUUM recovery performance
test on single relation.
I followed the advice below and applied the supplementary patch on top of V39:
Test-for-threshold.patch
This will ensure that we'll always enter the optimized path.
We're gonna use the threshold then as the relation size.

One idea could be to remove "nBlocksToInvalidate <
BUF_DROP_FULL_SCAN_THRESHOLD" part of check "if (cached &&
nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)" so that it
always use optimized path for the tests. Then use the relation size
as NBuffers/128, NBuffers/256, NBuffers/512 for different values of
shared buffers as 128MB, 1GB, 20GB, 100GB.

Each relation size is NBuffers/XXX, so I used the attached "rel.sh" script
to test from NBuffers/512 until NBuffers/8 relation size per shared_buffers.
I did not go further beyond 8 because it took too much time, and I could
already observe significant results until that.

[Vacuum Recovery Performance on Single Relation]
1. Setup synchronous streaming replication. I used the configuration
written at the bottom of this email.
2. [Primary] Create 1 table. (rel.sh create)
3. [Primary] Insert data of NBuffers/XXX size. Make sure to use the correct
size for the set shared_buffers by commenting out the right size in "insert"
of rel.sh script. (rel.sh insert)
4. [Primary] Delete table. (rel.sh delete)
5. [Standby] Optional: To double-check that DELETE is reflected on standby.
SELECT count(*) FROM tableXXX;
Make sure it returns 0.
6. [Standby] Pause WAL replay. (rel.sh pause)
(This script will execute SELECT pg_wal_replay_pause(); .)
7. [Primary] VACUUM the single relation. (rel.sh vacuum)
8. [Primary] After the vacuum finishes, stop the server. (rel.sh stop)
(The script will execute pg_ctl stop -D $PGDATA -w -mi)
9. [Standby] Resume WAL replay and promote the standby.
(rel.sh resume)
It basically prints a timestamp when resuming WAL replay,
and prints another timestamp when the promotion is done.
Compute the time difference.

[Results for VACUUM on single relation]
Average of 5 runs.

1. % REGRESSION
% Regression: (patched - master)/master

| rel_size | 128MB | 1GB | 20GB | 100GB |
|----------|--------|--------|--------|----------|
| NB/512 | 0.000% | 0.000% | 0.000% | -32.680% |
| NB/256 | 0.000% | 0.000% | 0.000% | 0.000% |
| NB/128 | 0.000% | 0.000% | 0.000% | -16.502% |
| NB/64 | 0.000% | 0.000% | 0.000% | -9.841% |
| NB/32 | 0.000% | 0.000% | 0.000% | -6.219% |
| NB/16 | 0.000% | 0.000% | 0.000% | 3.323% |
| NB/8 | 0.000% | 0.000% | 0.000% | 8.178% |

For 100GB shared_buffers, we can observe regression
beyond NBuffers/32. So with this, we can conclude
that NBuffers/32 is the right threshold.
For NBuffers/16 and beyond, the patched performs
worse than master. In other words, the cost of for finding
to be invalidated buffers gets higher in the optimized path
than the traditional path.

So in attached V39 patches, I have updated the threshold
BUF_DROP_FULL_SCAN_THRESHOLD to NBuffers/32.

2. [PATCHED]
Units: Seconds

| rel_size | 128MB | 1GB | 20GB | 100GB |
|----------|-------|-------|-------|-------|
| NB/512 | 0.106 | 0.106 | 0.106 | 0.206 |
| NB/256 | 0.106 | 0.106 | 0.106 | 0.306 |
| NB/128 | 0.106 | 0.106 | 0.206 | 0.506 |
| NB/64 | 0.106 | 0.106 | 0.306 | 0.907 |
| NB/32 | 0.106 | 0.106 | 0.406 | 1.508 |
| NB/16 | 0.106 | 0.106 | 0.706 | 3.109 |
| NB/8 | 0.106 | 0.106 | 1.307 | 6.614 |

3. MASTER
Units: Seconds

| rel_size | 128MB | 1GB | 20GB | 100GB |
|----------|-------|-------|-------|-------|
| NB/512 | 0.106 | 0.106 | 0.106 | 0.306 |
| NB/256 | 0.106 | 0.106 | 0.106 | 0.306 |
| NB/128 | 0.106 | 0.106 | 0.206 | 0.606 |
| NB/64 | 0.106 | 0.106 | 0.306 | 1.006 |
| NB/32 | 0.106 | 0.106 | 0.406 | 1.608 |
| NB/16 | 0.106 | 0.106 | 0.706 | 3.009 |
| NB/8 | 0.106 | 0.106 | 1.307 | 6.114 |

I used the following configurations:
[postgesql.conf]
shared_buffers = 100GB #20GB,1GB,128MB
autovacuum = off
full_page_writes = off
checkpoint_timeout = 30min
max_locks_per_transaction = 10000
max_wal_size = 20GB

# For streaming replication from primary. Don't uncomment on Standby.
synchronous_commit = remote_write
synchronous_standby_names = 'walreceiver'

# For Standby. Don't uncomment on Primary.
# hot_standby = on
#primary_conninfo = 'host=... user=... port=... application_name=walreceiver'

----------
B. Regression Test using the NBuffers/32 Threshold (V39 Patches)

For this one, we do NOT need the supplementary Test-for-threshold.patch.
Apply only the V39 patches.
But instead of using "rel.sh" test script, please use the attached "test.sh".
Similar to the tests I did before for 1000 relations, I executed the recovery
performance test, now with the threshold NBuffers/32.
The configuration setting in postgresql.conf is similar to the test above.

Each relation has 1 block, 8kB size. Total of 1000 relations.

Test procedures is almost similar to A, so I'll just summarize it,
1. Setup synchronous streaming replication and config settings.
2. [Primary] test.sh create
(The test.sh script will create 1000 tables)
3. [Primary] test.sh insert
4. [Primary] test.sh delete (Skip step 4-5 for TRUNCATE test)
5. [Standby] Optional for VACUUM test: To double-check that DELETE
is reflected on standby. SELECT count(*) FROM tableXXX;
Make sure it returns 0.
6. [Standby] test.sh pause
7. [Primary] "test.sh vacuum" for VACUUM test
"test,sh truncate" for TRUNCATE test
8. [Primary] If #7 is done, test.sh stop
9. [Standby] If primary is fully stopped, run "test.sh resume".
Compute the time difference.

[Results for VACUUM Recovery Performance for 1000 relations]
Unit is in seconds. Average of 5 executions.
% regression = (patched-master)/master

| s_b | Master | Patched | %reg |
|--------|--------|---------|---------|
| 128 MB | 0.306 | 0.306 | 0.00% |
| 1 GB | 0.506 | 0.306 | -39.53% |
| 20 GB | 14.522 | 0.306 | -97.89% |
| 100 GB | 66.564 | 0.306 | -99.54% |

[Results for TRUNCATE Recovery Performance for 1000 relations]
Unit is in seconds. Average of 5 executions.
% regression = (patched-master)/master

| s_b | Master | Patched | %reg |
|--------|--------|---------|---------|
| 128 MB | 0.206 | 0.206 | 0.00% |
| 1 GB | 0.506 | 0.206 | -59.29% |
| 20 GB | 16.476 | 0.206 | -98.75% |
| 100 GB | 88.261 | 0.206 | -99.77% |

The results for the patched were constant for all shared_buffers
settings for both TRUNCATE and VACUUM.
That means we can gain huge performance benefits with the patch.

The performance benefits have been tested a lot so there's no question
about that. So I think the final decision for value of threshold would come
if the results will be consistent with others. For now, in my test results,
the threshold NBuffers/32 is what I concluded. It's already indicated in
the attached V39 patch set.

[Specs Used]
Intel(R) Xeon(R) CPU E5-2637 v4 @ 3.50GHz
8 CPUs, 256GB Memory
XFS, RHEL7.2, latest Postgres(Head version)

Feedbacks are definitely welcome.
And if you want to test, I have already indicated the detailed steps
including the scripts I used. Have fun testing!

Regards,
Kirk Jamison

Attachments:

v39-0001-Optimize-DropRelFileNodeBuffers-for-recovery.patchapplication/octet-stream; name=v39-0001-Optimize-DropRelFileNodeBuffers-for-recovery.patchDownload
From e8e597ae711ac49ef8e767538516991c7cecf356 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 6 Jan 2021 06:05:56 +0000
Subject: [PATCH v39 1/3] Optimize DropRelFileNodeBuffers() for recovery.

The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool can be avoided when the number of blocks
to be truncated in a relation is below a certain threshold. For such cases,
we find the buffers by doing lookups in BufMapping table. This improves
the performance by more than 100 times in many cases for small tables
where the server is configured with a large value of shared buffers
(greater than 20GB).

This optimization helps cases (a) when vacuum or autovacuum truncated off
any of the empty pages at the end of a relation, or (b) when the relation is
truncated in the same transaction in which it was created.

This commit introduces a new API smgrnblocks_cached which returns a cached
value for the number of blocks in a relation fork. This helps us to determine
the exact size of relation which is required to apply this optimization. The
exact size is required to ensure that we don't leave any buffer for the
relation being dropped as otherwise the background writer or checkpointer
can lead to a PANIC error while flushing buffers corresponding to files that
don't exist.

Author: Kirk Jamison
Reviewed-by: Kyotaro Horiguchi, Takayuki Tsunakawa, and Amit Kapila
Tested-By: Haiying Tang
Discussion: https://postgr.es/m/OSBPR01MB3207DCA7EC725FDD661B3EDAEF660@OSBPR01MB3207.jpnprd01.prod.outlook.com
---
 src/backend/storage/buffer/bufmgr.c | 136 +++++++++++++++++++++++++++++++++---
 src/backend/storage/smgr/smgr.c     |  30 ++++++--
 src/include/storage/bufmgr.h        |   2 +-
 src/include/storage/smgr.h          |   1 +
 4 files changed, 154 insertions(+), 15 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 8f2c482..5711ff0 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -70,6 +70,14 @@
 
 #define RELS_BSEARCH_THRESHOLD		20
 
+/*
+ * This is the size (in the number of blocks) above which we scan the
+ * entire buffer pool to remove the buffers for all the pages of relation
+ * being dropped. For the relations with size below this threshold, we find
+ * the buffers by doing lookups in BufMapping table.
+ */
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32) (NBuffers / 32)
+
 typedef struct PrivateRefCountEntry
 {
 	Buffer		buffer;
@@ -473,6 +481,10 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
 static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FindAndDropRelFileNodeBuffers(RelFileNode rnode,
+										  ForkNumber forkNum,
+										  BlockNumber nForkBlock,
+										  BlockNumber firstDelBlock);
 static void AtProcExit_Buffers(int code, Datum arg);
 static void CheckForBufferLeaks(void);
 static int	rnode_comparator(const void *p1, const void *p2);
@@ -2965,19 +2977,19 @@ BufferGetLSNAtomic(Buffer buffer)
  *		later.  It is also the responsibility of higher-level code to ensure
  *		that no other process could be trying to load more pages of the
  *		relation into buffers.
- *
- *		XXX currently it sequentially searches the buffer pool, should be
- *		changed to more clever ways of searching.  However, this routine
- *		is used only in code paths that aren't very performance-critical,
- *		and we shouldn't slow down the hot paths to make it faster ...
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 	int			j;
+	RelFileNodeBackend rnode;
+	BlockNumber nForkBlock[MAX_FORKNUM];
+	BlockNumber nBlocksToInvalidate = 0;
+
+	rnode = smgr_reln->smgr_rnode;
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	if (RelFileNodeBackendIsTemp(rnode))
@@ -2991,6 +3003,56 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
 		return;
 	}
 
+	/*
+	 * To remove all the pages of the specified relation forks from the buffer
+	 * pool, we need to scan the entire buffer pool but we can optimize it by
+	 * finding the buffers from BufMapping table provided we know the exact
+	 * size of each fork of the relation. The exact size is required to ensure
+	 * that we don't leave any buffer for the relation being dropped as
+	 * otherwise the background writer or checkpointer can lead to a PANIC
+	 * error while flushing buffers corresponding to files that don't exist.
+	 *
+	 * To know the exact size, we rely on the size cached for each fork by us
+	 * during recovery which limits the optimization to recovery and on
+	 * standbys but we can easily extend it once we have shared cache for
+	 * relation size.
+	 *
+	 * In recovery, we cache the value returned by the first lseek(SEEK_END)
+	 * and the future writes keeps the cached value up-to-date. See
+	 * smgrextend. It is possible that the value of the first lseek is smaller
+	 * than the actual number of existing blocks in the file due to buggy
+	 * Linux kernels that might not have accounted for the recent write. But
+	 * that should be fine because there must not be any buffers after that
+	 * file size.
+	 */
+	for (i = 0; i < nforks; i++)
+	{
+		/* Get the number of blocks for a relation's fork */
+		nForkBlock[i] = smgrnblocks_cached(smgr_reln, forkNum[i]);
+
+		if (nForkBlock[i] == InvalidBlockNumber)
+		{
+			nBlocksToInvalidate = InvalidBlockNumber;
+			break;
+		}
+
+		/* calculate the number of blocks to be invalidated */
+		nBlocksToInvalidate += (nForkBlock[i] - firstDelBlock[i]);
+	}
+
+	/*
+	 * We apply the optimization iff the total number of blocks to invalidate
+	 * is below the BUF_DROP_FULL_SCAN_THRESHOLD.
+	 */
+	if (BlockNumberIsValid(nBlocksToInvalidate) &&
+		nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (j = 0; j < nforks; j++)
+			FindAndDropRelFileNodeBuffers(rnode.node, forkNum[j],
+										  nForkBlock[j], firstDelBlock[j]);
+		return;
+	}
+
 	for (i = 0; i < NBuffers; i++)
 	{
 		BufferDesc *bufHdr = GetBufferDescriptor(i);
@@ -3134,6 +3196,65 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 }
 
 /* ---------------------------------------------------------------------
+ *		FindAndDropRelFileNodeBuffers
+ *
+ *		This function performs look up in BufMapping table and removes from the
+ *		buffer pool all the pages of the specified relation fork that has block
+ *		number >= firstDelBlock. (In particular, with firstDelBlock = 0, all
+ *		pages are removed.)
+ * --------------------------------------------------------------------
+ */
+static void
+FindAndDropRelFileNodeBuffers(RelFileNode rnode, ForkNumber forkNum,
+							  BlockNumber nForkBlock,
+							  BlockNumber firstDelBlock)
+{
+	BlockNumber curBlock;
+
+	for (curBlock = firstDelBlock; curBlock < nForkBlock; curBlock++)
+	{
+		uint32		bufHash;	/* hash value for tag */
+		BufferTag	bufTag;		/* identity of requested block */
+		LWLock	   *bufPartitionLock;	/* buffer partition lock for it */
+		int			buf_id;
+		BufferDesc *bufHdr;
+		uint32		buf_state;
+
+		/* create a tag so we can lookup the buffer */
+		INIT_BUFFERTAG(bufTag, rnode, forkNum, curBlock);
+
+		/* determine its hash code and partition lock ID */
+		bufHash = BufTableHashCode(&bufTag);
+		bufPartitionLock = BufMappingPartitionLock(bufHash);
+
+		/* Check that it is in the buffer pool. If not, do nothing. */
+		LWLockAcquire(bufPartitionLock, LW_SHARED);
+		buf_id = BufTableLookup(&bufTag, bufHash);
+		LWLockRelease(bufPartitionLock);
+
+		if (buf_id < 0)
+			continue;
+
+		bufHdr = GetBufferDescriptor(buf_id);
+
+		/*
+		 * We need to lock the buffer header and recheck if the buffer is
+		 * still associated with the same block because the buffer could be
+		 * evicted by some other backend loading blocks for a different
+		 * relation after we release lock on the BufMapping table.
+		 */
+		buf_state = LockBufHdr(bufHdr);
+
+		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode) &&
+			bufHdr->tag.forkNum == forkNum &&
+			bufHdr->tag.blockNum >= firstDelBlock)
+			InvalidateBuffer(bufHdr);	/* releases spinlock */
+		else
+			UnlockBufHdr(bufHdr, buf_state);
+	}
+}
+
+/* ---------------------------------------------------------------------
  *		DropDatabaseBuffers
  *
  *		This function removes all the buffers in the buffer cache for a
@@ -3245,8 +3366,7 @@ PrintPinnedBufs(void)
  *		XXX currently it sequentially searches the buffer pool, should be
  *		changed to more clever ways of searching.  This routine is not
  *		used in any performance-critical code paths, so it's not worth
- *		adding additional overhead to normal paths to make it go faster;
- *		but see also DropRelFileNodeBuffers.
+ *		adding additional overhead to normal paths to make it go faster.
  * --------------------------------------------------------------------
  */
 void
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 0f31ff3..af603c3 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -549,6 +549,28 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 {
 	BlockNumber result;
 
+	/* Check and return if we get the cached value for the number of blocks. */
+	result = smgrnblocks_cached(reln, forknum);
+	if (result != InvalidBlockNumber)
+		return result;
+
+	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
+
+	reln->smgr_cached_nblocks[forknum] = result;
+
+	return result;
+}
+
+/*
+ *	smgrnblocks_cached() -- Get the cached number of blocks in the supplied
+ *							relation.
+ *
+ * Returns an InvalidBlockNumber when not in recovery and when the relation
+ * fork size is not cached.
+ */
+BlockNumber
+smgrnblocks_cached(SMgrRelation reln, ForkNumber forknum)
+{
 	/*
 	 * For now, we only use cached values in recovery due to lack of a shared
 	 * invalidation mechanism for changes in file size.
@@ -556,11 +578,7 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 	if (InRecovery && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
 		return reln->smgr_cached_nblocks[forknum];
 
-	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
-
-	reln->smgr_cached_nblocks[forknum] = result;
-
-	return result;
+	return InvalidBlockNumber;
 }
 
 /*
@@ -582,7 +600,7 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	 * Get rid of any buffers for the about-to-be-deleted blocks. bufmgr will
 	 * just drop them without bothering to write the contents.
 	 */
-	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
+	DropRelFileNodeBuffers(reln, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index ff6cd0f..0c484f3 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -203,7 +203,7 @@ extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index ebf4a19..a6fbf7b 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -99,6 +99,7 @@ extern void smgrwrite(SMgrRelation reln, ForkNumber forknum,
 extern void smgrwriteback(SMgrRelation reln, ForkNumber forknum,
 						  BlockNumber blocknum, BlockNumber nblocks);
 extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum);
+extern BlockNumber smgrnblocks_cached(SMgrRelation reln, ForkNumber forknum);
 extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
 						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
-- 
1.8.3.1

v39-0002-Optimize-DropRelFileNodesAllBuffers-for-recovery.patchapplication/octet-stream; name=v39-0002-Optimize-DropRelFileNodesAllBuffers-for-recovery.patchDownload
From fbe651fabed8f274b3bee573aae84ed300f249a4 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 6 Jan 2021 06:07:03 +0000
Subject: [PATCH v39 2/3] Optimize DropRelFileNodesAllBuffers() for recovery.

Similar to commit <>, this patch optimizes DropRelFileNodesAllBuffers() by
avoiding the complete buffer pool scan and instead find the buffers to be
invalidated by doing lookups in the BufMapping table.

This optimization helps operations where the relation files need to be
removed like Truncate, Drop, Abort of Create Table, etc.

Author: Kirk Jamison
Reviewed-by: Kyotaro Horiguchi, Takayuki Tsunakawa, and Amit Kapila
Tested-By: Haiying Tang
Discussion: https://postgr.es/m/OSBPR01MB3207DCA7EC725FDD661B3EDAEF660@OSBPR01MB3207.jpnprd01.prod.outlook.com
---
 src/backend/storage/buffer/bufmgr.c | 86 +++++++++++++++++++++++++++++++++----
 src/backend/storage/smgr/smgr.c     | 12 +++---
 src/include/storage/bufmgr.h        |  2 +-
 3 files changed, 84 insertions(+), 16 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 5711ff0..9b0e877 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -3104,28 +3104,33 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
+DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 {
-	int			i,
-				n = 0;
+	int			i;
+	int			j;
+	int			n = 0;
+	SMgrRelation	*rels;
+	BlockNumber (*block)[MAX_FORKNUM + 1];
+	BlockNumber	nBlocksToInvalidate = 0;
 	RelFileNode *nodes;
+	bool		cached = true;
 	bool		use_bsearch;
 
 	if (nnodes == 0)
 		return;
 
-	nodes = palloc(sizeof(RelFileNode) * nnodes);	/* non-local relations */
+	rels = palloc(sizeof(SMgrRelation) * nnodes);	/* non-local relations */
 
 	/* If it's a local relation, it's localbuf.c's problem. */
 	for (i = 0; i < nnodes; i++)
 	{
-		if (RelFileNodeBackendIsTemp(rnodes[i]))
+		if (RelFileNodeBackendIsTemp(smgr_reln[i]->smgr_rnode))
 		{
-			if (rnodes[i].backend == MyBackendId)
-				DropRelFileNodeAllLocalBuffers(rnodes[i].node);
+			if (smgr_reln[i]->smgr_rnode.backend == MyBackendId)
+				DropRelFileNodeAllLocalBuffers(smgr_reln[i]->smgr_rnode.node);
 		}
 		else
-			nodes[n++] = rnodes[i].node;
+			rels[n++] = smgr_reln[i];
 	}
 
 	/*
@@ -3134,10 +3139,72 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	 */
 	if (n == 0)
 	{
-		pfree(nodes);
+		pfree(rels);
+		return;
+	}
+
+	/*
+	 * This is used to remember the number of blocks for all the relations
+	 * forks.
+	 */
+	block = (BlockNumber (*)[MAX_FORKNUM + 1])
+			palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));
+
+	/*
+	 * We can avoid scanning the entire buffer pool if we know the exact size
+	 * of each of the given relation forks. See DropRelFileNodeBuffers.
+	 */
+	for (i = 0; i < n && cached; i++)
+	{
+		for (j = 0; j <= MAX_FORKNUM; j++)
+		{
+			/* Get the number of blocks for a relation's fork. */
+			block[i][j] = smgrnblocks_cached(rels[i], j);
+
+			/* We need to only consider the relation forks that exists. */
+			if (block[i][j] == InvalidBlockNumber)
+			{
+				if (!smgrexists(rels[i], j))
+					continue;
+				cached = false;
+				break;
+			}
+
+			/* calculate the total number of blocks to be invalidated */
+			nBlocksToInvalidate += block[i][j];
+		}
+	}
+
+	/*
+	 * We apply the optimization iff the total number of blocks to invalidate
+	 * is below the BUF_DROP_FULL_SCAN_THRESHOLD.
+	 */
+	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	{
+		for (i = 0; i < n; i++)
+		{
+			for (j = 0; j <= MAX_FORKNUM; j++)
+			{
+				/* ignore relation forks that doesn't exist */
+				if (!BlockNumberIsValid(block[i][j]))
+					continue;
+
+				/* drop all the buffers for a particular relation fork */
+				FindAndDropRelFileNodeBuffers(rels[i]->smgr_rnode.node,
+											  j, block[i][j], 0);
+			}
+		}
+
+		pfree(block);
+		pfree(rels);
 		return;
 	}
 
+	pfree(block);
+	nodes = palloc(sizeof(RelFileNode) * n); /* non-local relations */
+	for (i = 0; i < n; i++)
+		nodes[i] = rels[i]->smgr_rnode.node;
+
 	/*
 	 * For low number of relations to drop just use a simple walk through, to
 	 * save the bsearch overhead. The threshold to use is rather a guess than
@@ -3193,6 +3260,7 @@ DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes)
 	}
 
 	pfree(nodes);
+	pfree(rels);
 }
 
 /* ---------------------------------------------------------------------
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index af603c3..4dc2464 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -391,6 +391,12 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 		return;
 
 	/*
+	 * Get rid of any remaining buffers for the relations.  bufmgr will just
+	 * drop them without bothering to write the contents.
+	 */
+	DropRelFileNodesAllBuffers(rels, nrels);
+
+	/*
 	 * create an array which contains all relations to be dropped, and close
 	 * each relation's forks at the smgr level while at it
 	 */
@@ -408,12 +414,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	}
 
 	/*
-	 * Get rid of any remaining buffers for the relations.  bufmgr will just
-	 * drop them without bothering to write the contents.
-	 */
-	DropRelFileNodesAllBuffers(rnodes, nrels);
-
-	/*
 	 * It'd be nice to tell the stats collector to forget them immediately,
 	 * too. But we can't because we don't know the OIDs.
 	 */
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 0c484f3..fb00fda 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -205,7 +205,7 @@ extern void FlushRelationsAllBuffers(struct SMgrRelationData **smgrs, int nrels)
 extern void FlushDatabaseBuffers(Oid dbid);
 extern void DropRelFileNodeBuffers(struct SMgrRelationData *smgr_reln, ForkNumber *forkNum,
 								   int nforks, BlockNumber *firstDelBlock);
-extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
+extern void DropRelFileNodesAllBuffers(struct SMgrRelationData **smgr_reln, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
 #define RelationGetNumberOfBlocks(reln) \
-- 
1.8.3.1

v39-0003-Prevent-invalidating-blocks-in-smgrextend-during.patchapplication/octet-stream; name=v39-0003-Prevent-invalidating-blocks-in-smgrextend-during.patchDownload
From 396a69f89a49b3087d619ecc7fa39f637c4e3c34 Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 6 Jan 2021 06:30:29 +0000
Subject: [PATCH v39 3/3] Prevent invalidating blocks in smgrextend() during 
 recovery.

DropRelFileNodeBuffers relies on the behavior that cached blocks
returned by smgrnblocks() won't be invalidated by file extension
during recovery.
---
 src/backend/storage/smgr/smgr.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 4dc2464..fc05c3e 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -473,7 +473,14 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 	if (reln->smgr_cached_nblocks[forknum] == blocknum)
 		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
 	else
+	{
+		/*
+		 * DropRelFileNodeBuffers relies on the behavior that cached nblocks
+		 * won't be invalidated by file extension during recovery.
+		 */
+		Assert(!InRecovery);
 		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	}
 }
 
 /*
-- 
1.8.3.1

Test-for-threshold.patchapplication/octet-stream; name=Test-for-threshold.patchDownload
From c08570c9d33ecdb19ad88a61f5ba5eb442d16aca Mon Sep 17 00:00:00 2001
From: Kirk Jamison <k.jamison@jp.fujitsu.com>
Date: Wed, 6 Jan 2021 10:19:58 +0000
Subject: [PATCH] Test for threshold

This patch is for testing only. This patch should be applied
on top of the V39 patches.
We want to test for the approximate threshold wherein we can
see that the patched will perform worse than master.

In other words, our goal is to get the approximate threshold
and identify where the cost for finding to be invalidated
buffers gets higher in optimized path than the traditional path.
---
 src/backend/storage/buffer/bufmgr.c | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 9b0e877..77a841d 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -76,7 +76,7 @@
  * being dropped. For the relations with size below this threshold, we find
  * the buffers by doing lookups in BufMapping table.
  */
-#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32) (NBuffers / 32)
+//#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32) (NBuffers / 32)
 
 typedef struct PrivateRefCountEntry
 {
@@ -3044,8 +3044,7 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 	 * We apply the optimization iff the total number of blocks to invalidate
 	 * is below the BUF_DROP_FULL_SCAN_THRESHOLD.
 	 */
-	if (BlockNumberIsValid(nBlocksToInvalidate) &&
-		nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	if (BlockNumberIsValid(nBlocksToInvalidate))
 	{
 		for (j = 0; j < nforks; j++)
 			FindAndDropRelFileNodeBuffers(rnode.node, forkNum[j],
@@ -3179,7 +3178,7 @@ DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 	 * We apply the optimization iff the total number of blocks to invalidate
 	 * is below the BUF_DROP_FULL_SCAN_THRESHOLD.
 	 */
-	if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
+	if (cached)
 	{
 		for (i = 0; i < n; i++)
 		{
-- 
1.8.3.1

rel.shapplication/octet-stream; name=rel.shDownload
test.shapplication/octet-stream; name=test.shDownload
#254Tang, Haiying
tanghy.fnst@cn.fujitsu.com
In reply to: k.jamison@fujitsu.com (#253)
RE: [Patch] Optimize dropping of relation buffers using dlist

Hi Kirk,

And if you want to test, I have already indicated the detailed steps including the scripts I used. Have fun testing!

Thank you for your sharing of test steps and scripts. I'd like take a look at them and redo some of the tests using my machine. I'll send my test reults in a separate email after this.

Regards,
Tang

#255Tang, Haiying
tanghy.fnst@cn.fujitsu.com
In reply to: Tang, Haiying (#254)
RE: [Patch] Optimize dropping of relation buffers using dlist

I'd like take a look at them and redo some of the tests using my machine. I'll send my test reults in a separate email after this.

I did the same tests with Kirk's scripts using the latest patch on my own machine. The results look pretty good and similar with Kirk's.

average of 5 runs.

[VACUUM failover test for 1000 relations] Unit is second, %reg=(patched-master)/ master

| s_b | Master | Patched | %reg |
|--------------|---------------|--------------|--------------|
| 128 MB | 0.408 | 0.308 | -24.44% |
| 1 GB | 0.809 | 0.308 | -61.94% |
| 20 GB | 12.529 | 0.308 | -97.54% |
| 100 GB | 59.310 | 0.369 | -99.38% |

[TRUNCATE failover test for 1000 relations] Unit is second, %reg=(patched-master)/ master

| s_b | Master | Patched | %reg |
|--------------|---------------|--------------|--------------|
| 128 MB | 0.287 | 0.207 | -27.91% |
| 1 GB | 0.688 | 0.208 | -69.84% |
| 20 GB | 12.449 | 0.208 | -98.33% |
| 100 GB | 61.800 | 0.207 | -99.66% |

Besides, I did the test for threshold value again. (I rechecked my test process and found out that I forgot to check the data synchronization state on standby which may introduce some NOISE to my results.)
The following results show we can't get optimize over NBuffers/32 just like Kirk's test results, so I do approve with Kirk on the threshold value.

%regression:
| rel_size |128MB|1GB|20GB| 100GB |
|----------|----|----|----|-------|
| NB/512 | 0% | 0% | 0% | -48% |
| NB/256 | 0% | 0% | 0% | -33% |
| NB/128 | 0% | 0% | 0% | -9% |
| NB/64 | 0% | 0% | 0% | -5% |
| NB/32 | 0% | 0% |-4% | -3% |
| NB/16 | 0% | 0% |-4% | 1% |
| NB/8 | 1% | 0% | 1% | 3% |

Optimization details(unit: second):
patched:
shared_buffers NBuffers/512 NBuffers/256 NBuffers/128 NBuffers/64 NBuffers/32 NBuffers/16 NBuffers/8
-------------------------------------------------------------------------------------------------------------------------------------
128M 0.107 0.107 0.107 0.106 0.107 0.107 0.107
1G 0.107 0.107 0.107 0.107 0.107 0.107 0.107
20G 0.107 0.108 0.207 0.307 0.442 0.876 1.577
100G 0.208 0.308 0.559 1.060 1.961 4.567 7.922

master:
shared_buffers NBuffers/512 NBuffers/256 NBuffers/128 NBuffers/64 NBuffers/32 NBuffers/16 NBuffers/8
-------------------------------------------------------------------------------------------------------------------------------------
128M 0.107 0.107 0.107 0.107 0.107 0.107 0.106
1G 0.107 0.107 0.107 0.107 0.107 0.107 0.107
20G 0.107 0.107 0.208 0.308 0.457 0.910 1.560
100G 0.308 0.409 0.608 1.110 2.011 4.516 7.721

[Specs]
CPU : 40 processors (Intel(R) Xeon(R) Silver 4210 CPU @ 2.20GHz)
Memory: 128G
OS: CentOS 8

Any question to my test is welcome.

Regards,
Tang

#256Amit Kapila
amit.kapila16@gmail.com
In reply to: k.jamison@fujitsu.com (#253)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Jan 6, 2021 at 6:43 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

[Results for VACUUM on single relation]
Average of 5 runs.

1. % REGRESSION
% Regression: (patched - master)/master

| rel_size | 128MB | 1GB | 20GB | 100GB |
|----------|--------|--------|--------|----------|
| NB/512 | 0.000% | 0.000% | 0.000% | -32.680% |
| NB/256 | 0.000% | 0.000% | 0.000% | 0.000% |
| NB/128 | 0.000% | 0.000% | 0.000% | -16.502% |
| NB/64 | 0.000% | 0.000% | 0.000% | -9.841% |
| NB/32 | 0.000% | 0.000% | 0.000% | -6.219% |
| NB/16 | 0.000% | 0.000% | 0.000% | 3.323% |
| NB/8 | 0.000% | 0.000% | 0.000% | 8.178% |

For 100GB shared_buffers, we can observe regression
beyond NBuffers/32. So with this, we can conclude
that NBuffers/32 is the right threshold.
For NBuffers/16 and beyond, the patched performs
worse than master. In other words, the cost of for finding
to be invalidated buffers gets higher in the optimized path
than the traditional path.

So in attached V39 patches, I have updated the threshold
BUF_DROP_FULL_SCAN_THRESHOLD to NBuffers/32.

Thanks for the detailed tests. NBuffers/32 seems like an appropriate
value for the threshold based on these results. I would like to
slightly modify part of the commit message in the first patch as below
[1]: "The recovery path of DropRelFileNodeBuffers() is optimized so that scanning of the whole buffer pool can be avoided when the number of blocks to be truncated in a relation is below a certain threshold. For such cases, we find the buffers by doing lookups in BufMapping table. This improves the performance by more than 100 times in many cases when several small tables (tested with 1000 relations) are truncated and where the server is configured with a large value of shared buffers (greater than 100GB)."
has any more comments, I am planning to push the 0001 and 0002
sometime next week.

[1]: "The recovery path of DropRelFileNodeBuffers() is optimized so that scanning of the whole buffer pool can be avoided when the number of blocks to be truncated in a relation is below a certain threshold. For such cases, we find the buffers by doing lookups in BufMapping table. This improves the performance by more than 100 times in many cases when several small tables (tested with 1000 relations) are truncated and where the server is configured with a large value of shared buffers (greater than 100GB)."
"The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool can be avoided when the number of
blocks to be truncated in a relation is below a certain threshold. For
such cases, we find the buffers by doing lookups in BufMapping table.
This improves the performance by more than 100 times in many cases
when several small tables (tested with 1000 relations) are truncated
and where the server is configured with a large value of shared
buffers (greater than 100GB)."

--
With Regards,
Amit Kapila.

#257k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#256)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Thu, January 7, 2021 5:36 PM (JST), Amit Kapila wrote:

On Wed, Jan 6, 2021 at 6:43 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

[Results for VACUUM on single relation]
Average of 5 runs.

1. % REGRESSION
% Regression: (patched - master)/master

| rel_size | 128MB | 1GB | 20GB | 100GB |
|----------|--------|--------|--------|----------|
| NB/512 | 0.000% | 0.000% | 0.000% | -32.680% |
| NB/256 | 0.000% | 0.000% | 0.000% | 0.000% |
| NB/128 | 0.000% | 0.000% | 0.000% | -16.502% |
| NB/64 | 0.000% | 0.000% | 0.000% | -9.841% |
| NB/32 | 0.000% | 0.000% | 0.000% | -6.219% |
| NB/16 | 0.000% | 0.000% | 0.000% | 3.323% |
| NB/8 | 0.000% | 0.000% | 0.000% | 8.178% |

For 100GB shared_buffers, we can observe regression
beyond NBuffers/32. So with this, we can conclude
that NBuffers/32 is the right threshold.
For NBuffers/16 and beyond, the patched performs
worse than master. In other words, the cost of for finding
to be invalidated buffers gets higher in the optimized path
than the traditional path.

So in attached V39 patches, I have updated the threshold
BUF_DROP_FULL_SCAN_THRESHOLD to NBuffers/32.

Thanks for the detailed tests. NBuffers/32 seems like an appropriate
value for the threshold based on these results. I would like to
slightly modify part of the commit message in the first patch as below
[1], otherwise, I am fine with the changes. Unless you or anyone else
has any more comments, I am planning to push the 0001 and 0002
sometime next week.

[1]
"The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool can be avoided when the number of
blocks to be truncated in a relation is below a certain threshold. For
such cases, we find the buffers by doing lookups in BufMapping table.
This improves the performance by more than 100 times in many cases
when several small tables (tested with 1000 relations) are truncated
and where the server is configured with a large value of shared
buffers (greater than 100GB)."

Thank you for taking a look at the results of the tests. And it's also
consistent with the results from Tang too.
The commit message LGTM.

Regards,
Kirk Jamison

#258Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: k.jamison@fujitsu.com (#257)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Thu, 7 Jan 2021 09:25:22 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in

On Thu, January 7, 2021 5:36 PM (JST), Amit Kapila wrote:

On Wed, Jan 6, 2021 at 6:43 PM k.jamison@fujitsu.com
<k.jamison@fujitsu.com> wrote:

[Results for VACUUM on single relation]
Average of 5 runs.

1. % REGRESSION
% Regression: (patched - master)/master

| rel_size | 128MB | 1GB | 20GB | 100GB |
|----------|--------|--------|--------|----------|
| NB/512 | 0.000% | 0.000% | 0.000% | -32.680% |
| NB/256 | 0.000% | 0.000% | 0.000% | 0.000% |
| NB/128 | 0.000% | 0.000% | 0.000% | -16.502% |
| NB/64 | 0.000% | 0.000% | 0.000% | -9.841% |
| NB/32 | 0.000% | 0.000% | 0.000% | -6.219% |
| NB/16 | 0.000% | 0.000% | 0.000% | 3.323% |
| NB/8 | 0.000% | 0.000% | 0.000% | 8.178% |

For 100GB shared_buffers, we can observe regression
beyond NBuffers/32. So with this, we can conclude
that NBuffers/32 is the right threshold.
For NBuffers/16 and beyond, the patched performs
worse than master. In other words, the cost of for finding
to be invalidated buffers gets higher in the optimized path
than the traditional path.

So in attached V39 patches, I have updated the threshold
BUF_DROP_FULL_SCAN_THRESHOLD to NBuffers/32.

Thanks for the detailed tests. NBuffers/32 seems like an appropriate
value for the threshold based on these results. I would like to
slightly modify part of the commit message in the first patch as below
[1], otherwise, I am fine with the changes. Unless you or anyone else
has any more comments, I am planning to push the 0001 and 0002
sometime next week.

[1]
"The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool can be avoided when the number of
blocks to be truncated in a relation is below a certain threshold. For
such cases, we find the buffers by doing lookups in BufMapping table.
This improves the performance by more than 100 times in many cases
when several small tables (tested with 1000 relations) are truncated
and where the server is configured with a large value of shared
buffers (greater than 100GB)."

Thank you for taking a look at the results of the tests. And it's also
consistent with the results from Tang too.
The commit message LGTM.

+1.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#259Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#258)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Jan 8, 2021 at 7:03 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Thu, 7 Jan 2021 09:25:22 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in:

Thanks for the detailed tests. NBuffers/32 seems like an appropriate
value for the threshold based on these results. I would like to
slightly modify part of the commit message in the first patch as below
[1], otherwise, I am fine with the changes. Unless you or anyone else
has any more comments, I am planning to push the 0001 and 0002
sometime next week.

[1]
"The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool can be avoided when the number of
blocks to be truncated in a relation is below a certain threshold. For
such cases, we find the buffers by doing lookups in BufMapping table.
This improves the performance by more than 100 times in many cases
when several small tables (tested with 1000 relations) are truncated
and where the server is configured with a large value of shared
buffers (greater than 100GB)."

Thank you for taking a look at the results of the tests. And it's also
consistent with the results from Tang too.
The commit message LGTM.

+1.

I have pushed the 0001.

--
With Regards,
Amit Kapila.

#260Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Amit Kapila (#259)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Tue, 12 Jan 2021 08:49:53 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Fri, Jan 8, 2021 at 7:03 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Thu, 7 Jan 2021 09:25:22 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in:

Thanks for the detailed tests. NBuffers/32 seems like an appropriate
value for the threshold based on these results. I would like to
slightly modify part of the commit message in the first patch as below
[1], otherwise, I am fine with the changes. Unless you or anyone else
has any more comments, I am planning to push the 0001 and 0002
sometime next week.

[1]
"The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool can be avoided when the number of
blocks to be truncated in a relation is below a certain threshold. For
such cases, we find the buffers by doing lookups in BufMapping table.
This improves the performance by more than 100 times in many cases
when several small tables (tested with 1000 relations) are truncated
and where the server is configured with a large value of shared
buffers (greater than 100GB)."

Thank you for taking a look at the results of the tests. And it's also
consistent with the results from Tang too.
The commit message LGTM.

+1.

I have pushed the 0001.

Thank you for commiting this.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#261Amit Kapila
amit.kapila16@gmail.com
In reply to: Kyotaro Horiguchi (#260)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Wed, Jan 13, 2021 at 7:39 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Tue, 12 Jan 2021 08:49:53 +0530, Amit Kapila <amit.kapila16@gmail.com> wrote in

On Fri, Jan 8, 2021 at 7:03 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Thu, 7 Jan 2021 09:25:22 +0000, "k.jamison@fujitsu.com" <k.jamison@fujitsu.com> wrote in:

Thanks for the detailed tests. NBuffers/32 seems like an appropriate
value for the threshold based on these results. I would like to
slightly modify part of the commit message in the first patch as below
[1], otherwise, I am fine with the changes. Unless you or anyone else
has any more comments, I am planning to push the 0001 and 0002
sometime next week.

[1]
"The recovery path of DropRelFileNodeBuffers() is optimized so that
scanning of the whole buffer pool can be avoided when the number of
blocks to be truncated in a relation is below a certain threshold. For
such cases, we find the buffers by doing lookups in BufMapping table.
This improves the performance by more than 100 times in many cases
when several small tables (tested with 1000 relations) are truncated
and where the server is configured with a large value of shared
buffers (greater than 100GB)."

Thank you for taking a look at the results of the tests. And it's also
consistent with the results from Tang too.
The commit message LGTM.

+1.

I have pushed the 0001.

Thank you for commiting this.

Pushed 0002 as well.

--
With Regards,
Amit Kapila.

#262k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: Amit Kapila (#261)
RE: [Patch] Optimize dropping of relation buffers using dlist

On Wed, January 13, 2021 2:15 PM (JST), Amit Kapila wrote:

On Wed, Jan 13, 2021 at 7:39 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Tue, 12 Jan 2021 08:49:53 +0530, Amit Kapila
<amit.kapila16@gmail.com> wrote in

On Fri, Jan 8, 2021 at 7:03 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Thu, 7 Jan 2021 09:25:22 +0000, "k.jamison@fujitsu.com"

<k.jamison@fujitsu.com> wrote in:

Thanks for the detailed tests. NBuffers/32 seems like an
appropriate value for the threshold based on these results. I
would like to slightly modify part of the commit message in
the first patch as below [1], otherwise, I am fine with the
changes. Unless you or anyone else has any more comments, I am
planning to push the 0001 and 0002 sometime next week.

[1]
"The recovery path of DropRelFileNodeBuffers() is optimized so
that scanning of the whole buffer pool can be avoided when the
number of blocks to be truncated in a relation is below a
certain threshold. For such cases, we find the buffers by doing

lookups in BufMapping table.

This improves the performance by more than 100 times in many
cases when several small tables (tested with 1000 relations)
are truncated and where the server is configured with a large
value of shared buffers (greater than 100GB)."

Thank you for taking a look at the results of the tests. And
it's also consistent with the results from Tang too.
The commit message LGTM.

+1.

I have pushed the 0001.

Thank you for commiting this.

Pushed 0002 as well.

Thank you very much for committing those two patches, and for everyone here
who contributed in the simplifying the approaches, code reviews, testing, etc.

I compile with the --enable-coverage and check if the newly-added code and updated
parts were covered by tests.
Yes, the lines were hit including the updated lines of DropRelFileNodeBuffers(),
DropRelFileNodesAllBuffers(), smgrdounlinkall(), smgrnblocks().
Newly added APIs were covered too: FindAndDropRelFileNodeBuffers() and
smgrnblocks_cached().
However, the parts where UnlockBufHdr(bufHdr, buf_state); is called is not hit.
But I noticed that exists as well in previously existing functions in bufmgr.c.

Thank you very much again.

Regards,
Kirk Jamison

#263Thomas Munro
thomas.munro@gmail.com
In reply to: k.jamison@fujitsu.com (#262)
Re: [Patch] Optimize dropping of relation buffers using dlist

While rebasing CF #2933 (which drops the _cached stuff and makes this
optimisation always available, woo), I happened to notice that we're
summing the size of many relations and forks into a variable
nBlocksToInvalidate of type BlockNumber. That could overflow.

#264Amit Kapila
amit.kapila16@gmail.com
In reply to: Thomas Munro (#263)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Mar 12, 2021 at 4:58 AM Thomas Munro <thomas.munro@gmail.com> wrote:

While rebasing CF #2933 (which drops the _cached stuff and makes this
optimisation always available, woo), I happened to notice that we're
summing the size of many relations and forks into a variable
nBlocksToInvalidate of type BlockNumber. That could overflow.

I also think so. I think we have two ways to address that: (a) check
immediately after each time we add blocks to nBlocksToInvalidate to
see if it crosses the threshold value BUF_DROP_FULL_SCAN_THRESHOLD and
if so, then just break the loop; (b) change the variable type to
uint64.

Any better ideas?

--
With Regards,
Amit Kapila.

#265Thomas Munro
thomas.munro@gmail.com
In reply to: Amit Kapila (#264)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Mar 12, 2021 at 5:20 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

uint64

+1

#266tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Thomas Munro (#265)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Thomas Munro <thomas.munro@gmail.com>

On Fri, Mar 12, 2021 at 5:20 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

uint64

+1

+1
I'll send a patch later.

Regards
Takayuki Tsunakawa

#267tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Thomas Munro (#265)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Thomas Munro <thomas.munro@gmail.com>

uint64

+1

Thank you, the patch is attached (we tend to forget how large our world is... 64-bit) We're sorry to cause you trouble.

Regards
Takayuki Tsunakawa

Attachments:

v1-0001-Fix-overflow-when-counting-the-number-of-buffers-.patchapplication/octet-stream; name=v1-0001-Fix-overflow-when-counting-the-number-of-buffers-.patchDownload
From c1cb353108fcc2f24521089de26664861e52f9c5 Mon Sep 17 00:00:00 2001
From: Takayuki Tsunakawa <tsunakawa.takay@fujitsu.com>
Date: Fri, 12 Mar 2021 14:17:34 +0900
Subject: [PATCH v1] Fix overflow when counting the number of buffers to
 invalidate

---
 src/backend/storage/buffer/bufmgr.c | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 4c1d5ec..5aa7b9e 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -2987,7 +2987,7 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 	int			j;
 	RelFileNodeBackend rnode;
 	BlockNumber nForkBlock[MAX_FORKNUM];
-	BlockNumber nBlocksToInvalidate = 0;
+	uint64 		nBlocksToInvalidate = 0;
 
 	rnode = smgr_reln->smgr_rnode;
 
@@ -3111,7 +3111,7 @@ DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 	int			n = 0;
 	SMgrRelation *rels;
 	BlockNumber (*block)[MAX_FORKNUM + 1];
-	BlockNumber nBlocksToInvalidate = 0;
+	uint64 		nBlocksToInvalidate = 0;
 	RelFileNode *nodes;
 	bool		cached = true;
 	bool		use_bsearch;
-- 
2.10.1

#268Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#267)
Re: [Patch] Optimize dropping of relation buffers using dlist

At Fri, 12 Mar 2021 05:26:02 +0000, "tsunakawa.takay@fujitsu.com" <tsunakawa.takay@fujitsu.com> wrote in

From: Thomas Munro <thomas.munro@gmail.com>

uint64

+1

Thank you, the patch is attached (we tend to forget how large our world is... 64-bit) We're sorry to cause you trouble.

BUF_DROP_FULL_SCAN_THRESHOLD cannot be larger than the size of int
since Nbuffer is an int. but nBlocksToInvalidate being uint32 looks
somewhat too tight. So +1 for changing it to uint64.

We need fill all block[file][fork] array in DropRelFileNodesAllBuffers
so we cannot bailing out from the counting loop. We could do that
DropRelFileNodesAllBuffers but that doesn't seem effective so much.

So I vote for uint64 and not bailing out.

About the patch, it would be better to change the type of
BUF_DROP_FULL_SCAN_THRESHOLD to uint64, even though the current value
doesn't harm.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#269tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Kyotaro Horiguchi (#268)
1 attachment(s)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

About the patch, it would be better to change the type of
BUF_DROP_FULL_SCAN_THRESHOLD to uint64, even though the current
value
doesn't harm.

OK, attached, to be prepared for the distant future when NBuffers becomes 64-bit.

Regards
Takayuki Tsunakawa

Attachments:

v2-0001-Fix-overflow-when-counting-the-number-of-buffers-.patchapplication/octet-stream; name=v2-0001-Fix-overflow-when-counting-the-number-of-buffers-.patchDownload
From e34392fd548c496381073df95344ffae1ba1a0c6 Mon Sep 17 00:00:00 2001
From: Takayuki Tsunakawa <tsunakawa.takay@fujitsu.com>
Date: Fri, 12 Mar 2021 14:17:34 +0900
Subject: [PATCH v2] Fix overflow when counting the number of buffers to
 invalidate

---
 src/backend/storage/buffer/bufmgr.c | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 4c1d5ec..e4d257e 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -76,7 +76,7 @@
  * being dropped. For the relations with size below this threshold, we find
  * the buffers by doing lookups in BufMapping table.
  */
-#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint32) (NBuffers / 32)
+#define BUF_DROP_FULL_SCAN_THRESHOLD		(uint64) (NBuffers / 32)
 
 typedef struct PrivateRefCountEntry
 {
@@ -2987,7 +2987,7 @@ DropRelFileNodeBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
 	int			j;
 	RelFileNodeBackend rnode;
 	BlockNumber nForkBlock[MAX_FORKNUM];
-	BlockNumber nBlocksToInvalidate = 0;
+	uint64 		nBlocksToInvalidate = 0;
 
 	rnode = smgr_reln->smgr_rnode;
 
@@ -3111,7 +3111,7 @@ DropRelFileNodesAllBuffers(SMgrRelation *smgr_reln, int nnodes)
 	int			n = 0;
 	SMgrRelation *rels;
 	BlockNumber (*block)[MAX_FORKNUM + 1];
-	BlockNumber nBlocksToInvalidate = 0;
+	uint64 		nBlocksToInvalidate = 0;
 	RelFileNode *nodes;
 	bool		cached = true;
 	bool		use_bsearch;
-- 
2.10.1

#270k.jamison@fujitsu.com
k.jamison@fujitsu.com
In reply to: tsunakawa.takay@fujitsu.com (#269)
RE: [Patch] Optimize dropping of relation buffers using dlist

From: Tsunakawa, Takayuki/綱川 貴之 <tsunakawa.takay@fujitsu.com>
From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

About the patch, it would be better to change the type of
BUF_DROP_FULL_SCAN_THRESHOLD to uint64, even though the current

value

doesn't harm.

OK, attached, to be prepared for the distant future when NBuffers becomes
64-bit.

Thank you, Tsunakawa-san, for sending the quick fix. (I failed to notice to my thread.)

Regards,
Kirk Jamison

#271Amit Kapila
amit.kapila16@gmail.com
In reply to: tsunakawa.takay@fujitsu.com (#269)
Re: [Patch] Optimize dropping of relation buffers using dlist

On Fri, Mar 12, 2021 at 12:07 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:

From: Kyotaro Horiguchi <horikyota.ntt@gmail.com>

About the patch, it would be better to change the type of
BUF_DROP_FULL_SCAN_THRESHOLD to uint64, even though the current
value
doesn't harm.

OK, attached, to be prepared for the distant future when NBuffers becomes 64-bit.

Thanks for the patch. Pushed!

--
With Regards,
Amit Kapila.