On Tue, Nov 17, 2020 at 10:48 PM Amit Kapila <amit.kapil...@gmail.com> wrote:
> Yeah, it is good to verify VACUUM stuff but I have another question
> here. What about queries having functions that access the same
> relation (SELECT c1 FROM t1 WHERE c1 <= func(); assuming here function
> access the relation t1)? Now, here I think because the relation 't1'
> is already opened, it might use the same value of blocks from the
> shared cache even though the snapshot for relation 't1' when accessed
> from func() might be different. Am, I missing something, or is it
> dealt in some way?

I think it should be covered by the theory about implicit memory
barriers snapshots, but to simplify things I have now removed the
lock-free stuff from the main patch (0001), because it was a case of
premature optimisation and it distracted from the main concept.  The
main patch has 128-way partitioned LWLocks for the mapping table, and
then per-relfilenode spinlocks for modifying the size.  There are
still concurrency considerations, which I think are probably handled
with the dirty-update-wins algorithm you see in the patch.  In short:
due to extension and exclusive locks, size changes AKA dirty updates
are serialised, but clean updates can run concurrently, so we just
have to make sure that clean updates never clobber dirty updates -- do
you think that is on the right path?
From ac3c61926bf947a3288724bd02cf8439ff5c14bc Mon Sep 17 00:00:00 2001
From: Thomas Munro <thomas.mu...@gmail.com>
Date: Fri, 13 Nov 2020 14:38:41 +1300
Subject: [PATCH v2 1/2] WIP: Track relation sizes in shared memory.

Introduce a fixed size pool of SMgrSharedRelation objects.  A new GUC
smgr_shared_relation controls how many can exist at once, and they are
evicted as required.  "Dirty" SMgrSharedRelations can only be evicted
after being synced to disk.  Goals:

1.  Provide faster lookups of relation sizes, cutting down on lseek()
calls.  This supercedes the recovery-only caching added recently, and
replaces preexisting FSM and VM caching schemes.

2.  Stop trusting the operating system to keep track of the size of
files that we have recently extended, until fsync() has been called.

XXX smgrimmedsync() is maybe too blunt an instrument?

XXX perhaps mdsyncfiletag should tell smgr.c when it cleans forks via
some new interface, but it doesn't actually know if it's cleaning a fork
that extended a relation...

XXX perhaps bgwriter should try to clean them?

XXX currently reusing the array of locks also used for buffer mapping,
need to define some more in lwlocks.c...

Discussion: https://postgr.es/m/CAEepm%3D3SSw-Ty1DFcK%3D1rU-K6GSzYzfdD4d%2BZwapdN7dTa6%3DnQ%40mail.gmail.com
---
 contrib/pg_visibility/pg_visibility.c     |   1 -
 src/backend/access/heap/visibilitymap.c   |  28 +-
 src/backend/catalog/storage.c             |   2 -
 src/backend/storage/freespace/freespace.c |  35 +-
 src/backend/storage/ipc/ipci.c            |   3 +
 src/backend/storage/smgr/md.c             |  10 +
 src/backend/storage/smgr/smgr.c           | 500 ++++++++++++++++++++--
 src/backend/utils/misc/guc.c              |  11 +
 src/include/storage/smgr.h                |  17 +-
 9 files changed, 518 insertions(+), 89 deletions(-)

diff --git a/contrib/pg_visibility/pg_visibility.c b/contrib/pg_visibility/pg_visibility.c
index 54e47b810f..702951e487 100644
--- a/contrib/pg_visibility/pg_visibility.c
+++ b/contrib/pg_visibility/pg_visibility.c
@@ -392,7 +392,6 @@ pg_truncate_visibility_map(PG_FUNCTION_ARGS)
 	check_relation_relkind(rel);
 
 	RelationOpenSmgr(rel);
-	rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = InvalidBlockNumber;
 
 	block = visibilitymap_prepare_truncate(rel, 0);
 	if (BlockNumberIsValid(block))
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index b1072183bc..8fe29ecae7 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -547,6 +547,7 @@ static Buffer
 vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 {
 	Buffer		buf;
+	BlockNumber nblocks;
 
 	/*
 	 * We might not have opened the relation at the smgr level yet, or we
@@ -557,20 +558,12 @@ vm_readbuf(Relation rel, BlockNumber blkno, bool extend)
 	 */
 	RelationOpenSmgr(rel);
 
-	/*
-	 * If we haven't cached the size of the visibility map fork yet, check it
-	 * first.
-	 */
-	if (rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber)
-	{
-		if (smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-			smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
-		else
-			rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] = 0;
-	}
+	if (!smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
+		smgrcreate(rel->rd_smgr, VISIBILITYMAP_FORKNUM, false);
+	nblocks = smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
 
 	/* Handle requests beyond EOF */
-	if (blkno >= rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM])
+	if (blkno >= nblocks)
 	{
 		if (extend)
 			vm_extend(rel, blkno + 1);
@@ -636,17 +629,10 @@ vm_extend(Relation rel, BlockNumber vm_nblocks)
 	/* Might have to re-open if a cache flush happened */
 	RelationOpenSmgr(rel);
 
-	/*
-	 * Create the file first if it doesn't exist.  If smgr_vm_nblocks is
-	 * positive then it must exist, no need for an smgrexists call.
-	 */
-	if ((rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == 0 ||
-		 rel->rd_smgr->smgr_cached_nblocks[VISIBILITYMAP_FORKNUM] == InvalidBlockNumber) &&
-		!smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
+	/* * Create the file first if it doesn't exist. */
+	if (!smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
 		smgrcreate(rel->rd_smgr, VISIBILITYMAP_FORKNUM, false);
 
-	/* 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);
 
 	/* Now extend the file */
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index d538f25726..45f62030aa 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -290,8 +290,6 @@ 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;
 
 	/* Prepare for truncation of MAIN fork of the relation */
 	forks[nforks] = MAIN_FORKNUM;
diff --git a/src/backend/storage/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 6a96126b0c..629923c4d4 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -531,29 +531,18 @@ static Buffer
 fsm_readbuf(Relation rel, FSMAddress addr, bool extend)
 {
 	BlockNumber blkno = fsm_logical_to_physical(addr);
+	BlockNumber nblocks;
 	Buffer		buf;
 
 	RelationOpenSmgr(rel);
 
-	/*
-	 * If we haven't cached the size of the FSM yet, check it first.  Also
-	 * recheck if the requested block seems to be past end, since our cached
-	 * value might be stale.  (We send smgr inval messages on truncation, but
-	 * not on extension.)
-	 */
-	if (rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] == InvalidBlockNumber ||
-		blkno >= rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM])
-	{
-		/* 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);
-		else
-			rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] = 0;
-	}
+	if (smgrexists(rel->rd_smgr, FSM_FORKNUM))
+		nblocks = smgrnblocks(rel->rd_smgr, FSM_FORKNUM);
+	else
+		nblocks = 0;
 
 	/* Handle requests beyond EOF */
-	if (blkno >= rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM])
+	if (blkno >= nblocks)
 	{
 		if (extend)
 			fsm_extend(rel, blkno + 1);
@@ -621,18 +610,10 @@ fsm_extend(Relation rel, BlockNumber fsm_nblocks)
 	/* Might have to re-open if a cache flush happened */
 	RelationOpenSmgr(rel);
 
-	/*
-	 * Create the FSM file first if it doesn't exist.  If
-	 * smgr_cached_nblocks[FSM_FORKNUM] is positive then it must exist, no
-	 * need for an smgrexists call.
-	 */
-	if ((rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] == 0 ||
-		 rel->rd_smgr->smgr_cached_nblocks[FSM_FORKNUM] == InvalidBlockNumber) &&
-		!smgrexists(rel->rd_smgr, FSM_FORKNUM))
+	/* Create the FSM file first if it doesn't exist. */
+	if (!smgrexists(rel->rd_smgr, FSM_FORKNUM))
 		smgrcreate(rel->rd_smgr, FSM_FORKNUM, false);
 
-	/* 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);
 
 	while (fsm_nblocks_now < fsm_nblocks)
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index 96c2aaabbd..cfc55d3691 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -121,6 +121,7 @@ CreateSharedMemoryAndSemaphores(void)
 		size = add_size(size, hash_estimate_size(SHMEM_INDEX_SIZE,
 												 sizeof(ShmemIndexEnt)));
 		size = add_size(size, dsm_estimate_size());
+		size = add_size(size, smgr_shmem_size());
 		size = add_size(size, BufferShmemSize());
 		size = add_size(size, LockShmemSize());
 		size = add_size(size, PredicateLockShmemSize());
@@ -212,6 +213,8 @@ CreateSharedMemoryAndSemaphores(void)
 
 	dsm_shmem_init();
 
+	smgr_shmem_init();
+
 	/*
 	 * Set up xlog, clog, and buffers
 	 */
diff --git a/src/backend/storage/smgr/md.c b/src/backend/storage/smgr/md.c
index 1d4aa482cc..6228447e31 100644
--- a/src/backend/storage/smgr/md.c
+++ b/src/backend/storage/smgr/md.c
@@ -1332,6 +1332,16 @@ mdsyncfiletag(const FileTag *ftag, char *path)
 		need_to_close = true;
 	}
 
+	/*
+	 * XXX: We could have an interface smgrbeginclean() that would return true
+	 * if it has managed to set SR_SYNC and clean SR_JUST_DIRTIED, and then if
+	 * so, after our sync runs we could call smgrfinishclean() with our
+	 * success/failure report, which would clear SR_DIRTY if SR_JUST_DIRTIED
+	 * hasn't been set in the meantime.  But... how can we know if *this*
+	 * segment is one that represents an extension?  SR_DIRTY is just
+	 * interested in syncing extended blocks.
+	 */
+
 	/* Sync the file. */
 	result = FileSync(file, WAIT_EVENT_DATA_FILE_SYNC);
 	save_errno = errno;
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dcc09df0c7..b5dc370bba 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -19,13 +19,65 @@
 
 #include "access/xlog.h"
 #include "lib/ilist.h"
+#include "port/atomics.h"
+#include "port/pg_bitutils.h"
 #include "storage/bufmgr.h"
+#include "storage/shmem.h"
 #include "storage/ipc.h"
+#include "storage/lwlock.h"
 #include "storage/md.h"
 #include "storage/smgr.h"
 #include "utils/hsearch.h"
 #include "utils/inval.h"
 
+/*
+ * An entry in the hash table that allows us to look up objects in the
+ * SMgrSharedRelation pool by rnode (+ backend).
+ */
+typedef struct SMgrSharedRelationMapping
+{
+	RelFileNodeBackend rnode;
+	int				index;
+} SMgrSharedRelationMapping;
+
+/*
+ * An object in shared memory tracks the size of the forks of a relation.
+ */
+struct SMgrSharedRelation
+{
+	RelFileNodeBackend rnode;
+	BlockNumber		nblocks[MAX_FORKNUM + 1];
+	pg_atomic_uint32 flags;
+};
+
+/* For now, we borrow the buffer managers array of locks.  XXX fixme */
+#define SR_PARTITIONS NUM_BUFFER_PARTITIONS
+#define SR_PARTITION_LOCK(hash) (&MainLWLockArray[BUFFER_MAPPING_LWLOCK_OFFSET].lock)
+
+/* Flags. */
+#define SR_LOCKED					0x01
+#define SR_VALID					0x02
+
+/* Each forknum gets its own dirty, syncing and just dirtied bits. */
+#define SR_DIRTY(forknum)			(0x04 << ((forknum) + (MAX_FORKNUM + 1) * 0))
+#define SR_SYNCING(forknum)			(0x04 << ((forknum) + (MAX_FORKNUM + 1) * 1))
+#define SR_JUST_DIRTIED(forknum)	(0x04 << ((forknum) + (MAX_FORKNUM + 1) * 2))
+
+/* Masks to test if any forknum is currently dirty or syncing. */
+#define SR_SYNCING_MASK				(((SR_SYNCING(MAX_FORKNUM + 1) - 1) ^ (SR_SYNCING(0) - 1)))
+#define SR_DIRTY_MASK				(((SR_DIRTY(MAX_FORKNUM + 1) - 1) ^ (SR_DIRTY(0) - 1)))
+
+/* Extract the lowest dirty forknum from flags (there must be at least one). */
+#define SR_GET_ONE_DIRTY(mask)		pg_rightmost_one_pos32((((mask) >> 2) & (SR_DIRTY_MASK >> 2)))
+
+typedef struct SMgrSharedRelationPool
+{
+	pg_atomic_uint32 next;
+	SMgrSharedRelation objects[FLEXIBLE_ARRAY_MEMBER];
+} SMgrSharedRelationPool;
+
+static SMgrSharedRelationPool *sr_pool;
+static HTAB *sr_mapping_table;
 
 /*
  * This struct of function pointers defines the API between smgr.c and
@@ -98,6 +150,408 @@ static dlist_head unowned_relns;
 /* local function prototypes */
 static void smgrshutdown(int code, Datum arg);
 
+/* GUCs. */
+int smgr_shared_relations = 1000;
+
+/*
+ * Try to get the size of a relation's fork by looking it up in the mapping
+ * table with a shared lock.  This will succeed if the SMgrRelation already
+ * exists.
+ */
+static BlockNumber
+smgrnblocks_shared(SMgrRelation reln, ForkNumber forknum)
+{
+	SMgrSharedRelationMapping *mapping;
+	SMgrSharedRelation *sr;
+	uint32	hash;
+	LWLock *mapping_lock;
+	BlockNumber result = InvalidBlockNumber;
+
+	hash = get_hash_value(sr_mapping_table, &reln->smgr_rnode);
+	mapping_lock = SR_PARTITION_LOCK(hash);
+
+	LWLockAcquire(mapping_lock, LW_SHARED);
+	mapping = hash_search_with_hash_value(sr_mapping_table,
+										  &reln->smgr_rnode,
+										  hash,
+										  HASH_FIND,
+										  NULL);
+	if (mapping)
+	{
+		sr = &sr_pool->objects[mapping->index];
+		result = sr->nblocks[forknum];
+	}
+	LWLockRelease(mapping_lock);
+
+	return result;
+}
+
+/*
+ * Lock a SMgrSharedRelation.  The lock is a spinlock that should be held for
+ * only a few instructions.  The return value is the current set of flags,
+ * which may be modified and then passed to smgr_unlock_sr() to be atomically
+ * when the lock is released.
+ */
+static uint32
+smgr_lock_sr(SMgrSharedRelation *sr)
+{
+
+	for (;;)
+	{
+		uint32	old_flags = pg_atomic_read_u32(&sr->flags);
+		uint32	flags;
+
+		if (!(old_flags & SR_LOCKED))
+		{
+			flags = old_flags | SR_LOCKED;
+			if (pg_atomic_compare_exchange_u32(&sr->flags, &old_flags, flags))
+				return flags;
+		}
+	}
+	return 0; /* unreachable */
+}
+
+/*
+ * Unlock a SMgrSharedRelation, atomically updating its flags at the same
+ * time.
+ */
+static void
+smgr_unlock_sr(SMgrSharedRelation *sr, uint32 flags)
+{
+	pg_write_barrier();
+	pg_atomic_write_u32(&sr->flags, flags & ~SR_LOCKED);
+}
+
+/*
+ * Allocate a new invalid SMgrSharedRelation, and return it locked.
+ *
+ * The replacement algorithm is a simple FIFO design with no second chance for
+ * now.
+ */
+static SMgrSharedRelation *
+smgr_alloc_sr(void)
+{
+	SMgrSharedRelationMapping *mapping;
+	SMgrSharedRelation *sr;
+	uint32 index;
+	LWLock *mapping_lock;
+	uint32 flags;
+	RelFileNodeBackend rnode;
+	uint32 hash;
+
+ retry:
+	/* Lock the next one in clock-hand order. */
+	index = pg_atomic_fetch_add_u32(&sr_pool->next, 1) % smgr_shared_relations;
+	sr = &sr_pool->objects[index];
+	flags = smgr_lock_sr(sr);
+
+	/* If it's unused, can return it, still locked, immediately. */
+	if (!(flags & SR_VALID))
+		return sr;
+
+	/*
+	 * Copy the rnode and unlock.  We'll briefly acquire both mapping and SR
+	 * locks, but we need to do it in that order, so we'll unlock the SR
+	 * first.
+	 */
+	rnode = sr->rnode;
+	smgr_unlock_sr(sr, flags);
+
+	hash = get_hash_value(sr_mapping_table, &rnode);
+	mapping_lock = SR_PARTITION_LOCK(hash);
+
+	LWLockAcquire(mapping_lock, LW_EXCLUSIVE);
+	mapping = hash_search_with_hash_value(sr_mapping_table,
+										  &rnode,
+										  hash,
+										  HASH_FIND,
+										  NULL);
+	if (!mapping || mapping->index != index)
+	{
+		/* Too slow, it's gone or now points somewhere else.  Go around. */
+		LWLockRelease(mapping_lock);
+		goto retry;
+	}
+
+	/* We will lock the SR for just a few instructions. */
+	flags = smgr_lock_sr(sr);
+	Assert(flags & SR_VALID);
+
+	/*
+	 * If another backend is currently syncing any fork, we aren't allowed to
+	 * evict it, and waiting for it would be pointless because that other
+	 * backend already plans to allocate it.  So go around.
+	 */
+	if (flags & SR_SYNCING_MASK)
+	{
+		smgr_unlock_sr(sr, flags);
+		LWLockRelease(mapping_lock);
+		goto retry;
+	}
+
+	/*
+	 * We will sync every fork that is dirty, and then we'll try to
+	 * evict it.
+	 */
+	while (flags & SR_DIRTY_MASK)
+	{
+		SMgrRelation reln;
+		ForkNumber forknum = SR_GET_ONE_DIRTY(flags);
+
+		/* Set the sync bit, clear the just-dirtied bit and unlock. */
+		flags |= SR_SYNCING(forknum);
+		flags &= ~SR_JUST_DIRTIED(forknum);
+		smgr_unlock_sr(sr, flags);
+		LWLockRelease(mapping_lock);
+
+		/*
+		 * Perform the I/O, with no locks held.
+		 * XXX It sucks that we fsync every segment, not just the ones that need it...
+		 */
+		reln = smgropen(rnode.node, rnode.backend);
+		smgrimmedsync(reln, forknum);
+
+		/*
+		 * Reacquire the locks.  The object can't have been evicted,
+		 * because we set a sync bit.
+		 * XXX And what if it's dropped?
+		 */
+		LWLockAcquire(mapping_lock, LW_EXCLUSIVE);
+		flags = smgr_lock_sr(sr);
+		Assert(flags & SR_SYNCING(forknum));
+		if (flags & SR_JUST_DIRTIED(forknum))
+		{
+			/*
+			 * Someone else dirtied it while we were syncing, so we can't mark
+			 * it clean.  Let's give up on this SR and go around again.
+			 */
+			smgr_unlock_sr(sr, flags);
+			LWLockRelease(mapping_lock);
+			goto retry;
+		}
+
+		/* This fork is clean! */
+		flags &= ~SR_SYNCING(forknum);
+		flags &= ~SR_DIRTY(forknum);
+	}
+
+	/*
+	 * If we made it this far, there are no dirty forks, so we're now allowed
+	 * to evict the SR from the pool and the mapping table.
+	 */
+	flags &= ~SR_VALID;
+	smgr_unlock_sr(sr, flags);
+
+	/* Remove from the mapping table. */
+	hash_search_with_hash_value(sr_mapping_table,
+								&rnode,
+								hash,
+								HASH_REMOVE,
+								NULL);
+	LWLockRelease(mapping_lock);
+
+	/*
+	 * XXX: We unlock while doing HASH_REMOVE on principle.  Maybe it'd be OK
+	 * to hold it now that the clock hand is far away and there is no way
+	 * anyone can look up this SR through buffer mapping table.
+	 */
+	flags = smgr_lock_sr(sr);
+	if (flags & SR_VALID)
+	{
+		/* Oops, someone else got it. */
+		smgr_unlock_sr(sr, flags);
+		goto retry;
+	}
+
+	return sr;
+}
+
+/*
+ * Set the number of blocks in a relation, in shared memory, and optionally
+ * also mark the relation as "dirty" (meaning the it must be fsync'd before it
+ * can be evicted).
+ */
+static void
+smgrnblocks_update(SMgrRelation reln,
+				   ForkNumber forknum,
+				   BlockNumber nblocks,
+				   bool mark_dirty)
+{
+	SMgrSharedRelationMapping *mapping;
+	SMgrSharedRelation *sr = NULL;
+	uint32		hash;
+	LWLock *mapping_lock;
+	uint32 flags;
+
+	hash = get_hash_value(sr_mapping_table, &reln->smgr_rnode);
+	mapping_lock = SR_PARTITION_LOCK(hash);
+
+ retry:
+	LWLockAcquire(mapping_lock, LW_SHARED);
+	mapping = hash_search_with_hash_value(sr_mapping_table,
+										  &reln->smgr_rnode,
+										  hash,
+										  HASH_FIND,
+										  NULL);
+	if (mapping)
+	{
+		sr = &sr_pool->objects[mapping->index];
+		flags = smgr_lock_sr(sr);
+		if (mark_dirty)
+		{
+			/*
+			 * Extend and truncate clobber the value, and there are no races
+			 * to worry about because they can have higher level exclusive
+			 * locking on the relation.
+			 */
+			sr->nblocks[forknum] = nblocks;
+
+			/*
+			 * Mark it dirty, and if it's currently being sync'd, make sure it
+			 * stays dirty after that completes.
+			 */
+			flags |= SR_DIRTY(forknum);
+			if (flags & SR_SYNCING(forknum))
+				flags |= SR_JUST_DIRTIED(forknum);
+		}
+		else if (!(flags & SR_DIRTY(forknum)))
+		{
+			/*
+			 * We won't clobber a dirty value with a non-dirty update, to
+			 * avoid races against concurrent extend/truncate, but we can
+			 * install a new clean value.
+			 */
+			sr->nblocks[forknum] = nblocks;
+		}
+		smgr_unlock_sr(sr, flags);
+	}
+	LWLockRelease(mapping_lock);
+
+	/* If we didn't find it, then we'll need to allocate one. */
+	if (!sr)
+	{
+		bool found;
+
+		sr = smgr_alloc_sr();
+
+		/* Upgrade to exclusive lock so we can create a mapping. */
+		LWLockAcquire(mapping_lock, LW_EXCLUSIVE);
+		mapping = hash_search_with_hash_value(sr_mapping_table,
+											  &reln->smgr_rnode,
+											  hash,
+											  HASH_ENTER,
+											  &found);
+		if (!found)
+		{
+			/* Success!  Initialize. */
+			mapping->index = sr - sr_pool->objects;
+			smgr_unlock_sr(sr, SR_VALID);
+			sr->rnode = reln->smgr_rnode;
+			for (int i = 0; i <= MAX_FORKNUM; ++i)
+				sr->nblocks[i] = InvalidBlockNumber;
+			LWLockRelease(mapping_lock);
+		}
+		else
+		{
+			/* Someone beat us to it.  Go around again. */
+			smgr_unlock_sr(sr, 0);		/* = not valid */
+			LWLockRelease(mapping_lock);
+			goto retry;
+		}
+	}
+}
+
+static void
+smgr_drop_sr(SMgrRelation reln)
+{
+	SMgrSharedRelationMapping *mapping;
+	SMgrSharedRelation *sr;
+	uint32	hash;
+	LWLock *mapping_lock;
+	uint32 flags;
+
+	hash = get_hash_value(sr_mapping_table, &reln->smgr_rnode);
+	mapping_lock = SR_PARTITION_LOCK(hash);
+
+	LWLockAcquire(mapping_lock, LW_EXCLUSIVE);
+	mapping = hash_search_with_hash_value(sr_mapping_table,
+										  &reln->smgr_rnode,
+										  hash,
+										  HASH_FIND,
+										  NULL);
+	if (mapping)
+	{
+		sr = &sr_pool->objects[mapping->index];
+
+	retry:
+		flags = smgr_lock_sr(sr);
+		Assert(flags & SR_VALID);
+
+		if (flags & SR_SYNCING_MASK)
+		{
+			/*
+			 * Oops, someone's syncing one of its forks; nothing to do but
+			 * wait.
+			 */
+			smgr_unlock_sr(sr, flags);
+			pg_usleep(100000);
+			goto retry;
+		}
+
+		/* Mark it invalid and drop the mapping. */
+		smgr_unlock_sr(sr, ~SR_VALID);
+		hash_search_with_hash_value(sr_mapping_table,
+									&reln->smgr_rnode,
+									hash,
+									HASH_REMOVE,
+									NULL);
+	}
+	LWLockRelease(mapping_lock);
+}
+
+size_t
+smgr_shmem_size(void)
+{
+	size_t size = 0;
+
+	size = add_size(size,
+					sizeof(offsetof(SMgrSharedRelationPool, objects) +
+						   sizeof(SMgrSharedRelation) * smgr_shared_relations));
+	size = add_size(size,
+					hash_estimate_size(smgr_shared_relations,
+									   sizeof(SMgrSharedRelationMapping)));
+
+	return size;
+}
+
+void
+smgr_shmem_init(void)
+{
+	HASHCTL		info;
+	bool found;
+
+	info.keysize = sizeof(RelFileNodeBackend);
+	info.entrysize = sizeof(SMgrSharedRelationMapping);
+	info.num_partitions = SR_PARTITIONS;
+	sr_mapping_table = ShmemInitHash("SMgrSharedRelation Mapping Table",
+									 smgr_shared_relations,
+									 smgr_shared_relations,
+									 &info,
+									 HASH_ELEM | HASH_BLOBS | HASH_PARTITION);
+
+	sr_pool = ShmemInitStruct("SMgrSharedRelation Pool",
+							  offsetof(SMgrSharedRelationPool, objects) +
+							  sizeof(SMgrSharedRelation) * smgr_shared_relations,
+							  &found);
+	if (!found)
+	{
+		pg_atomic_init_u32(&sr_pool->next, 0);
+		for (uint32 i = 0; i < smgr_shared_relations; ++i)
+		{
+			pg_atomic_init_u32(&sr_pool->objects[i].flags, 0);
+		}
+	}
+}
 
 /*
  *	smgrinit(), smgrshutdown() -- Initialize or shut down storage
@@ -175,8 +629,6 @@ smgropen(RelFileNode rnode, BackendId backend)
 		/* hash_search already filled in the lookup key */
 		reln->smgr_owner = NULL;
 		reln->smgr_targblock = InvalidBlockNumber;
-		for (int i = 0; i <= MAX_FORKNUM; ++i)
-			reln->smgr_cached_nblocks[i] = InvalidBlockNumber;
 		reln->smgr_which = 0;	/* we only have md.c at present */
 
 		/* implementation-specific initialization */
@@ -247,6 +699,9 @@ smgrclearowner(SMgrRelation *owner, SMgrRelation reln)
 bool
 smgrexists(SMgrRelation reln, ForkNumber forknum)
 {
+	if (smgrnblocks_shared(reln, forknum) != InvalidBlockNumber)
+		return true;
+
 	return smgrsw[reln->smgr_which].smgr_exists(reln, forknum);
 }
 
@@ -430,6 +885,9 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	for (i = 0; i < nrels; i++)
 		CacheInvalidateSmgr(rnodes[i]);
 
+	for (i = 0; i < nrels; i++)
+		smgr_drop_sr(rels[i]);
+
 	/*
 	 * Delete the physical file(s).
 	 *
@@ -465,16 +923,7 @@ smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum,
 {
 	smgrsw[reln->smgr_which].smgr_extend(reln, forknum, blocknum,
 										 buffer, skipFsync);
-
-	/*
-	 * Normally we expect this to increase nblocks by one, but if the cached
-	 * value isn't as expected, just invalidate it so the next call asks the
-	 * kernel.
-	 */
-	if (reln->smgr_cached_nblocks[forknum] == blocknum)
-		reln->smgr_cached_nblocks[forknum] = blocknum + 1;
-	else
-		reln->smgr_cached_nblocks[forknum] = InvalidBlockNumber;
+	smgrnblocks_update(reln, forknum, blocknum + 1, true);
 }
 
 /*
@@ -550,16 +999,16 @@ smgrnblocks(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 && reln->smgr_cached_nblocks[forknum] != InvalidBlockNumber)
-		return reln->smgr_cached_nblocks[forknum];
+	/* Can we get the answer from shared memory with only a share lock? */
+	result = smgrnblocks_shared(reln, forknum);
+	if (result != InvalidBlockNumber)
+		return result;
 
+	/* Ask the kernel. */
 	result = smgrsw[reln->smgr_which].smgr_nblocks(reln, forknum);
 
-	reln->smgr_cached_nblocks[forknum] = result;
+	/* Update the value in shared memory for faster service next time. */
+	smgrnblocks_update(reln, forknum, result, false);
 
 	return result;
 }
@@ -600,19 +1049,8 @@ smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nb
 	/* Do the truncation */
 	for (i = 0; i < nforks; i++)
 	{
-		/* Make the cached size is invalid if we encounter an error. */
-		reln->smgr_cached_nblocks[forknum[i]] = InvalidBlockNumber;
-
 		smgrsw[reln->smgr_which].smgr_truncate(reln, forknum[i], nblocks[i]);
-
-		/*
-		 * We might as well update the local smgr_cached_nblocks values. The
-		 * smgr cache inval message that this function sent will cause other
-		 * backends to invalidate their copies of smgr_fsm_nblocks and
-		 * smgr_vm_nblocks, and these ones too at the next command boundary.
-		 * But these ensure they aren't outright wrong until then.
-		 */
-		reln->smgr_cached_nblocks[forknum[i]] = nblocks[i];
+		smgrnblocks_update(reln, forknum[i], nblocks[i], true);
 	}
 }
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index bb34630e8e..175ac555e6 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -82,6 +82,7 @@
 #include "storage/pg_shmem.h"
 #include "storage/predicate.h"
 #include "storage/proc.h"
+#include "storage/smgr.h"
 #include "storage/standby.h"
 #include "tcop/tcopprot.h"
 #include "tsearch/ts_cache.h"
@@ -2445,6 +2446,16 @@ static struct config_int ConfigureNamesInt[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"smgr_shared_relations", PGC_POSTMASTER, RESOURCES_MEM,
+			gettext_noop("Sets the number of shared relation objects in memory at one time."),
+			NULL
+		},
+		&smgr_shared_relations,
+		1000, 64, INT_MAX,
+		NULL, NULL, NULL
+	},
+
 	/*
 	 * See also CheckRequiredParameterValues() if this parameter changes
 	 */
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index f28a842401..2eea69213b 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -18,6 +18,13 @@
 #include "storage/block.h"
 #include "storage/relfilenode.h"
 
+/* GUCs. */
+extern int smgr_shared_relations;
+
+/* Definition private to smgr.c. */
+struct SMgrSharedRelation;
+typedef struct SMgrSharedRelation SMgrSharedRelation;
+
 /*
  * smgr.c maintains a table of SMgrRelation objects, which are essentially
  * cached file handles.  An SMgrRelation is created (if not already present)
@@ -44,14 +51,7 @@ typedef struct SMgrRelationData
 	/* pointer to owning pointer, or NULL if none */
 	struct SMgrRelationData **smgr_owner;
 
-	/*
-	 * The following fields are reset to InvalidBlockNumber upon a cache flush
-	 * event, and hold the last known size for each fork.  This information is
-	 * currently only reliable during recovery, since there is no cache
-	 * invalidation for fork extension.
-	 */
 	BlockNumber smgr_targblock; /* current insertion target block */
-	BlockNumber smgr_cached_nblocks[MAX_FORKNUM + 1];	/* last known size */
 
 	/* additional public fields may someday exist here */
 
@@ -77,6 +77,9 @@ typedef SMgrRelationData *SMgrRelation;
 #define SmgrIsTemp(smgr) \
 	RelFileNodeBackendIsTemp((smgr)->smgr_rnode)
 
+extern size_t smgr_shmem_size(void);
+extern void smgr_shmem_init(void);
+
 extern void smgrinit(void);
 extern SMgrRelation smgropen(RelFileNode rnode, BackendId backend);
 extern bool smgrexists(SMgrRelation reln, ForkNumber forknum);
-- 
2.20.1

From eaf9efd4df49e62b991c7c7347613f64c6269336 Mon Sep 17 00:00:00 2001
From: Thomas Munro <thomas.mu...@gmail.com>
Date: Thu, 19 Nov 2020 17:09:51 +1300
Subject: [PATCH v2 2/2] WIP: Provide a lock-free fast path for smgrnblocks().

SMgrRelation objects gain a pointer to the last known SMgrSharedRelation
object.  There are three concurrency hazards to worry about:

1.  The SMgrSharedRelation pointed to could be evicted at any time.  We
record a generation number, and insert memory barriers so that we can
detect that and fall back to a slower path.

2.  The nblocks value is read without any locking, which is atomic
because it is a 32 bit value, and PostgreSQL requires atomic 32 bit
reads generally.

3.  The nblocks value must be free enough for scans, extension,
truncatation and dropping buffers, because the those operations all
executed memory barriers when it acquired a snapshot to scan (which
doesn't need to see blocks added after that) or an exclusive heavyweight
lock to extend, truncate or drop.  XXX right?

XXX That's the idea anyway, but this is just a sketch; almost certainly
incomplet and inkorrect.

Discussion: https://postgr.es/m/CAEepm%3D3SSw-Ty1DFcK%3D1rU-K6GSzYzfdD4d%2BZwapdN7dTa6%3DnQ%40mail.gmail.com
---
 src/backend/storage/smgr/smgr.c | 58 +++++++++++++++++++++++++++++++--
 src/include/storage/smgr.h      |  4 +++
 2 files changed, 60 insertions(+), 2 deletions(-)

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index b5dc370bba..c96fa46785 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -48,6 +48,7 @@ struct SMgrSharedRelation
 	RelFileNodeBackend rnode;
 	BlockNumber		nblocks[MAX_FORKNUM + 1];
 	pg_atomic_uint32 flags;
+	pg_atomic_uint64 generation;		/* mapping change */
 };
 
 /* For now, we borrow the buffer managers array of locks.  XXX fixme */
@@ -153,6 +154,40 @@ static void smgrshutdown(int code, Datum arg);
 /* GUCs. */
 int smgr_shared_relations = 1000;
 
+/*
+ * Try to get the size of a relation's fork without locking.
+ */
+static BlockNumber
+smgrnblocks_fast(SMgrRelation reln, ForkNumber forknum)
+{
+	SMgrSharedRelation *sr = reln->smgr_shared;
+	BlockNumber result;
+
+	if (sr)
+	{
+		pg_read_barrier();
+
+		/* We can load int-sized values atomically without special measures. */
+		Assert(sizeof(sr->nblocks[forknum]) == sizeof(uint32));
+		result = sr->nblocks[forknum];
+
+		/*
+		 * With a read barrier between the loads, we can check that the object
+		 * still refers to the same rnode before trusting the answer.
+		 */
+		pg_read_barrier();
+		if (pg_atomic_read_u64(&sr->generation) == reln->smgr_shared_generation)
+			return result;
+
+		/*
+		 * The generation doesn't match, the shared relation must have been
+		 * evicted since we got a pointer to it.  We'll need to do more work.
+		 */
+	}
+
+	return InvalidBlockNumber;
+}
+
 /*
  * Try to get the size of a relation's fork by looking it up in the mapping
  * table with a shared lock.  This will succeed if the SMgrRelation already
@@ -180,6 +215,10 @@ smgrnblocks_shared(SMgrRelation reln, ForkNumber forknum)
 	{
 		sr = &sr_pool->objects[mapping->index];
 		result = sr->nblocks[forknum];
+
+		/* We can take the fast path until this SR is eventually evicted. */
+		reln->smgr_shared = sr;
+		reln->smgr_shared_generation = pg_atomic_read_u64(&sr->generation);
 	}
 	LWLockRelease(mapping_lock);
 
@@ -337,9 +376,14 @@ smgr_alloc_sr(void)
 
 	/*
 	 * If we made it this far, there are no dirty forks, so we're now allowed
-	 * to evict the SR from the pool and the mapping table.
+	 * to evict the SR from the pool and the mapping table.  Make sure that
+	 * smgrnblocks_fast() sees that its pointer is now invalid by bumping the
+	 * generation.
 	 */
 	flags &= ~SR_VALID;
+	pg_atomic_write_u64(&sr->generation,
+						pg_atomic_read_u64(&sr->generation) + 1);
+	pg_write_barrier();
 	smgr_unlock_sr(sr, flags);
 
 	/* Remove from the mapping table. */
@@ -447,6 +491,8 @@ smgrnblocks_update(SMgrRelation reln,
 			mapping->index = sr - sr_pool->objects;
 			smgr_unlock_sr(sr, SR_VALID);
 			sr->rnode = reln->smgr_rnode;
+			pg_atomic_write_u64(&sr->generation,
+								pg_atomic_read_u64(&sr->generation) + 1);
 			for (int i = 0; i <= MAX_FORKNUM; ++i)
 				sr->nblocks[i] = InvalidBlockNumber;
 			LWLockRelease(mapping_lock);
@@ -549,6 +595,7 @@ smgr_shmem_init(void)
 		for (uint32 i = 0; i < smgr_shared_relations; ++i)
 		{
 			pg_atomic_init_u32(&sr_pool->objects[i].flags, 0);
+			pg_atomic_init_u64(&sr_pool->objects[i].generation, 0);
 		}
 	}
 }
@@ -629,6 +676,8 @@ smgropen(RelFileNode rnode, BackendId backend)
 		/* hash_search already filled in the lookup key */
 		reln->smgr_owner = NULL;
 		reln->smgr_targblock = InvalidBlockNumber;
+		reln->smgr_shared = NULL;
+		reln->smgr_shared_generation = 0;
 		reln->smgr_which = 0;	/* we only have md.c at present */
 
 		/* implementation-specific initialization */
@@ -699,7 +748,7 @@ smgrclearowner(SMgrRelation *owner, SMgrRelation reln)
 bool
 smgrexists(SMgrRelation reln, ForkNumber forknum)
 {
-	if (smgrnblocks_shared(reln, forknum) != InvalidBlockNumber)
+	if (smgrnblocks_fast(reln, forknum) != InvalidBlockNumber)
 		return true;
 
 	return smgrsw[reln->smgr_which].smgr_exists(reln, forknum);
@@ -999,6 +1048,11 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
 {
 	BlockNumber result;
 
+	/* Can we get the answer from shared memory without locking? */
+	result = smgrnblocks_fast(reln, forknum);
+	if (result != InvalidBlockNumber)
+		return result;
+
 	/* Can we get the answer from shared memory with only a share lock? */
 	result = smgrnblocks_shared(reln, forknum);
 	if (result != InvalidBlockNumber)
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index 2eea69213b..b2cab478a2 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -51,6 +51,10 @@ typedef struct SMgrRelationData
 	/* pointer to owning pointer, or NULL if none */
 	struct SMgrRelationData **smgr_owner;
 
+	/* pointer to shared object, valid if non-NULL and generation matches */
+	SMgrSharedRelation *smgr_shared;
+	uint64		smgr_shared_generation;
+
 	BlockNumber smgr_targblock; /* current insertion target block */
 
 	/* additional public fields may someday exist here */
-- 
2.20.1

Reply via email to