diff --git a/contrib/pg_visibility/pg_visibility.c b/contrib/pg_visibility/pg_visibility.c
index 1372bb6..4d342ea 100644
--- a/contrib/pg_visibility/pg_visibility.c
+++ b/contrib/pg_visibility/pg_visibility.c
@@ -383,6 +383,8 @@ pg_truncate_visibility_map(PG_FUNCTION_ARGS)
 {
 	Oid			relid = PG_GETARG_OID(0);
 	Relation	rel;
+	ForkNumber	fork;
+	BlockNumber	block;
 
 	rel = relation_open(relid, AccessExclusiveLock);
 
@@ -392,20 +394,26 @@ pg_truncate_visibility_map(PG_FUNCTION_ARGS)
 	RelationOpenSmgr(rel);
 	rel->rd_smgr->smgr_vm_nblocks = InvalidBlockNumber;
 
-	visibilitymap_truncate(rel, 0);
-
-	if (RelationNeedsWAL(rel))
+	block = visibilitymap_truncate_prepare(rel, 0);
+	if (BlockNumberIsValid(block))
 	{
-		xl_smgr_truncate xlrec;
+		fork = VISIBILITYMAP_FORKNUM;
+		smgrtruncate(rel->rd_smgr, &fork, 1, &block);
+
+		if (RelationNeedsWAL(rel))
+		{
+			xl_smgr_truncate xlrec;
 
-		xlrec.blkno = 0;
-		xlrec.rnode = rel->rd_node;
-		xlrec.flags = SMGR_TRUNCATE_VM;
+			xlrec.blkno = 0;
+			xlrec.rnode = rel->rd_node;
+			xlrec.flags = SMGR_TRUNCATE_VM;
 
-		XLogBeginInsert();
-		XLogRegisterData((char *) &xlrec, sizeof(xlrec));
+			XLogBeginInsert();
+			XLogRegisterData((char *) &xlrec, sizeof(xlrec));
 
-		XLogInsert(RM_SMGR_ID, XLOG_SMGR_TRUNCATE | XLR_SPECIAL_REL_UPDATE);
+			XLogInsert(RM_SMGR_ID, XLOG_SMGR_TRUNCATE |
+					   XLR_SPECIAL_REL_UPDATE);
+		}
 	}
 
 	/*
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index a08922b..351fc31 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -17,7 +17,7 @@
  *		visibilitymap_set	 - set a bit in a previously pinned page
  *		visibilitymap_get_status - get status of bits
  *		visibilitymap_count  - count number of bits set in visibility map
- *		visibilitymap_truncate	- truncate the visibility map
+ *		visibilitymap_truncate_prepare - truncate only tail bits of map pages
  *
  * NOTES
  *
@@ -430,16 +430,18 @@ visibilitymap_count(Relation rel, BlockNumber *all_visible, BlockNumber *all_fro
 }
 
 /*
- *	visibilitymap_truncate - truncate the visibility map
+ *	visibilitymap_truncate_prepare - truncate only tail bits of map page
+ *									 and return the block number for actual
+ *									 truncation later
  *
- * The caller must hold AccessExclusiveLock on the relation, to ensure that
- * other backends receive the smgr invalidation event that this function sends
- * before they access the VM again.
+ * Note that this does not truncate the actual visibility map pages.
+ * When this function is called, the caller must eventually follow it with
+ * smgrtruncate() call to actually truncate visibility map pages.
  *
  * nheapblocks is the new size of the heap.
  */
-void
-visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
+BlockNumber
+visibilitymap_truncate_prepare(Relation rel, BlockNumber nheapblocks)
 {
 	BlockNumber newnblocks;
 
@@ -459,7 +461,7 @@ visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
 	 * nothing to truncate.
 	 */
 	if (!smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM))
-		return;
+		return InvalidBlockNumber;
 
 	/*
 	 * Unless the new size is exactly at a visibility map page boundary, the
@@ -480,7 +482,7 @@ visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
 		if (!BufferIsValid(mapBuffer))
 		{
 			/* nothing to do, the file was already smaller */
-			return;
+			return InvalidBlockNumber;
 		}
 
 		page = BufferGetPage(mapBuffer);
@@ -528,20 +530,10 @@ visibilitymap_truncate(Relation rel, BlockNumber nheapblocks)
 	if (smgrnblocks(rel->rd_smgr, VISIBILITYMAP_FORKNUM) <= newnblocks)
 	{
 		/* nothing to do, the file was already smaller than requested size */
-		return;
+		return InvalidBlockNumber;
 	}
-
-	/* Truncate the unused VM pages, and send smgr inval message */
-	smgrtruncate(rel->rd_smgr, VISIBILITYMAP_FORKNUM, newnblocks);
-
-	/*
-	 * We might as well update the local smgr_vm_nblocks setting. smgrtruncate
-	 * sent an smgr cache inval message, which will cause other backends to
-	 * invalidate their copy of smgr_vm_nblocks, and this one too at the next
-	 * command boundary.  But this ensures it isn't outright wrong until then.
-	 */
-	if (rel->rd_smgr)
-		rel->rd_smgr->smgr_vm_nblocks = newnblocks;
+	else
+		return newnblocks;
 }
 
 /*
diff --git a/src/backend/catalog/storage.c b/src/backend/catalog/storage.c
index 3cc886f..623cf9f 100644
--- a/src/backend/catalog/storage.c
+++ b/src/backend/catalog/storage.c
@@ -231,6 +231,10 @@ RelationTruncate(Relation rel, BlockNumber nblocks)
 {
 	bool		fsm;
 	bool		vm;
+	ForkNumber	forks[MAX_FORKNUM];
+	BlockNumber	blocks[MAX_FORKNUM];
+	BlockNumber	first_removed_nblocks = InvalidBlockNumber;
+	int		nforks = 0;
 
 	/* Open it at the smgr level if not already done */
 	RelationOpenSmgr(rel);
@@ -242,15 +246,33 @@ RelationTruncate(Relation rel, BlockNumber nblocks)
 	rel->rd_smgr->smgr_fsm_nblocks = InvalidBlockNumber;
 	rel->rd_smgr->smgr_vm_nblocks = InvalidBlockNumber;
 
-	/* Truncate the FSM first if it exists */
+	/* Find the dirty FSM page and return a block number. */
 	fsm = smgrexists(rel->rd_smgr, FSM_FORKNUM);
 	if (fsm)
-		FreeSpaceMapTruncateRel(rel, nblocks);
+	{
+		blocks[nforks] = FreeSpaceMapLocateBlock(rel, nblocks);
+		if (BlockNumberIsValid(blocks[nforks]))
+		{
+			first_removed_nblocks = nblocks;
+			forks[nforks] = FSM_FORKNUM;
+			nforks++;
+		}
+	}
 
-	/* Truncate the visibility map too if it exists. */
+	/*
+	 * Truncate only the tail bits of VM and return the block number
+	 * for actual truncation later in smgrtruncate.
+	 */
 	vm = smgrexists(rel->rd_smgr, VISIBILITYMAP_FORKNUM);
 	if (vm)
-		visibilitymap_truncate(rel, nblocks);
+	{
+		blocks[nforks] = visibilitymap_truncate_prepare(rel, nblocks);
+		if (BlockNumberIsValid(blocks[nforks]))
+		{
+			forks[nforks] = VISIBILITYMAP_FORKNUM;
+			nforks++;
+		}
+	}
 
 	/*
 	 * We WAL-log the truncation before actually truncating, which means
@@ -290,8 +312,22 @@ RelationTruncate(Relation rel, BlockNumber nblocks)
 			XLogFlush(lsn);
 	}
 
-	/* Do the real work */
-	smgrtruncate(rel->rd_smgr, MAIN_FORKNUM, nblocks);
+	/* Pinpoint the MAIN fork and its blocks */
+	forks[nforks] = MAIN_FORKNUM;
+	blocks[nforks] = nblocks;
+	nforks++;
+
+	/* Truncate relation forks simultaneously */
+	smgrtruncate(rel->rd_smgr, forks, nforks, blocks);
+
+	/*
+	 * Update upper-level FSM pages to account for the truncation.
+	 * This is important because the just-truncated pages were likely
+	 * marked as all-free, and would be preferentially selected.
+	 */
+	if (fsm)
+		FreeSpaceMapVacuumRange(rel, first_removed_nblocks,
+								InvalidBlockNumber);
 }
 
 /*
@@ -588,6 +624,13 @@ smgr_redo(XLogReaderState *record)
 		xl_smgr_truncate *xlrec = (xl_smgr_truncate *) XLogRecGetData(record);
 		SMgrRelation reln;
 		Relation	rel;
+		ForkNumber	forks[MAX_FORKNUM];
+		BlockNumber	blocks[MAX_FORKNUM];
+		BlockNumber	first_removed_nblocks = InvalidBlockNumber;
+		int		nforks = 0;
+		bool		fsm = false;
+		bool		main = false;
+		bool		vm = false;
 
 		reln = smgropen(xlrec->rnode, InvalidBackendId);
 
@@ -616,23 +659,60 @@ smgr_redo(XLogReaderState *record)
 		 */
 		XLogFlush(lsn);
 
+		/*
+		 * To speedup recovery, we identify the about-to-be-truncated blocks
+		 * of relation forks first, then truncate those simultaneously later.
+		 */
 		if ((xlrec->flags & SMGR_TRUNCATE_HEAP) != 0)
 		{
-			smgrtruncate(reln, MAIN_FORKNUM, xlrec->blkno);
-
-			/* Also tell xlogutils.c about it */
-			XLogTruncateRelation(xlrec->rnode, MAIN_FORKNUM, xlrec->blkno);
+			forks[nforks] = MAIN_FORKNUM;
+			blocks[nforks] = xlrec->blkno;
+			nforks++;
+			main = true;
 		}
 
-		/* Truncate FSM and VM too */
 		rel = CreateFakeRelcacheEntry(xlrec->rnode);
 
 		if ((xlrec->flags & SMGR_TRUNCATE_FSM) != 0 &&
 			smgrexists(reln, FSM_FORKNUM))
-			FreeSpaceMapTruncateRel(rel, xlrec->blkno);
+		{
+			blocks[nforks] = FreeSpaceMapLocateBlock(rel, xlrec->blkno);
+			if (BlockNumberIsValid(blocks[nforks]))
+			{
+				first_removed_nblocks = xlrec->blkno;
+				forks[nforks] = FSM_FORKNUM;
+				nforks++;
+				fsm = true;
+			}
+		}
 		if ((xlrec->flags & SMGR_TRUNCATE_VM) != 0 &&
 			smgrexists(reln, VISIBILITYMAP_FORKNUM))
-			visibilitymap_truncate(rel, xlrec->blkno);
+		{
+			blocks[nforks] = visibilitymap_truncate_prepare(rel, xlrec->blkno);
+			if (BlockNumberIsValid(blocks[nforks]))
+			{
+				forks[nforks] = VISIBILITYMAP_FORKNUM;
+				nforks++;
+				vm = true;
+			}
+		}
+
+		/* Truncate relation forks simultaneously */
+		if (main || fsm || vm)
+			smgrtruncate(reln, forks, nforks, blocks);
+
+		/* Also tell xlogutils.c about it */
+		if (main)
+			XLogTruncateRelation(xlrec->rnode, MAIN_FORKNUM, xlrec->blkno);
+
+		/*
+		 * Update upper-level FSM pages to account for the truncation.
+		 * This is important because the just-truncated pages were likely
+		 * marked as all-free, and would be preferentially selected.
+		 */
+		if (fsm)
+			FreeSpaceMapVacuumRange(rel, first_removed_nblocks,
+									InvalidBlockNumber);
 
 		FreeFakeRelcacheEntry(rel);
 	}
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index 6f3a402..1f2b600 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -2900,8 +2900,8 @@ BufferGetLSNAtomic(Buffer buffer)
 /* ---------------------------------------------------------------------
  *		DropRelFileNodeBuffers
  *
- *		This function removes from the buffer pool all the pages of the
- *		specified relation fork that have block numbers >= firstDelBlock.
+ *		This function simultaneously removes from the buffer pool all the
+ *		pages of the relation forks that have block numbers >= firstDelBlock.
  *		(In particular, with firstDelBlock = 0, all pages are removed.)
  *		Dirty pages are simply dropped, without bothering to write them
  *		out first.  Therefore, this is NOT rollback-able, and so should be
@@ -2924,8 +2924,8 @@ BufferGetLSNAtomic(Buffer buffer)
  * --------------------------------------------------------------------
  */
 void
-DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber forkNum,
-					   BlockNumber firstDelBlock)
+DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+					   int nforks, BlockNumber *firstDelBlock)
 {
 	int			i;
 
@@ -2933,7 +2933,12 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber forkNum,
 	if (RelFileNodeBackendIsTemp(rnode))
 	{
 		if (rnode.backend == MyBackendId)
-			DropRelFileNodeLocalBuffers(rnode.node, forkNum, firstDelBlock);
+		{
+			int		j;
+			for (j = 0; j < nforks; j++)
+				DropRelFileNodeLocalBuffers(rnode.node, forkNum[j],
+											firstDelBlock[j]);
+		}
 		return;
 	}
 
@@ -2941,6 +2946,7 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber forkNum,
 	{
 		BufferDesc *bufHdr = GetBufferDescriptor(i);
 		uint32		buf_state;
+		int		j = 0;
 
 		/*
 		 * We can make this a tad faster by prechecking the buffer tag before
@@ -2962,11 +2968,18 @@ DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber forkNum,
 			continue;
 
 		buf_state = LockBufHdr(bufHdr);
-		if (RelFileNodeEquals(bufHdr->tag.rnode, rnode.node) &&
-			bufHdr->tag.forkNum == forkNum &&
-			bufHdr->tag.blockNum >= firstDelBlock)
-			InvalidateBuffer(bufHdr);	/* releases spinlock */
-		else
+
+		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/freespace/freespace.c b/src/backend/storage/freespace/freespace.c
index 2383094..473c2e5 100644
--- a/src/backend/storage/freespace/freespace.c
+++ b/src/backend/storage/freespace/freespace.c
@@ -247,16 +247,17 @@ GetRecordedFreeSpace(Relation rel, BlockNumber heapBlk)
 }
 
 /*
- * FreeSpaceMapTruncateRel - adjust for truncation of a relation.
+ * FreeSpaceMapLocateBlock - adjust for truncation of a relation.
  *
- * The caller must hold AccessExclusiveLock on the relation, to ensure that
- * other backends receive the smgr invalidation event that this function sends
- * before they access the FSM again.
+ * This function finds the dirty page and returns a block number.
+ *
+ * The caller of this function must eventually call smgrtruncate() to actually
+ * truncate FSM pages.
  *
  * nblocks is the new size of the heap.
  */
-void
-FreeSpaceMapTruncateRel(Relation rel, BlockNumber nblocks)
+BlockNumber
+FreeSpaceMapLocateBlock(Relation rel, BlockNumber nblocks)
 {
 	BlockNumber new_nfsmblocks;
 	FSMAddress	first_removed_address;
@@ -270,7 +271,7 @@ FreeSpaceMapTruncateRel(Relation rel, BlockNumber nblocks)
 	 * truncate.
 	 */
 	if (!smgrexists(rel->rd_smgr, FSM_FORKNUM))
-		return;
+		return InvalidBlockNumber;
 
 	/* Get the location in the FSM of the first removed heap block */
 	first_removed_address = fsm_get_location(nblocks, &first_removed_slot);
@@ -285,7 +286,7 @@ FreeSpaceMapTruncateRel(Relation rel, BlockNumber nblocks)
 	{
 		buf = fsm_readbuf(rel, first_removed_address, false);
 		if (!BufferIsValid(buf))
-			return;				/* nothing to do; the FSM was already smaller */
+			return InvalidBlockNumber;	/* nothing to do; the FSM was already smaller */
 		LockBuffer(buf, BUFFER_LOCK_EXCLUSIVE);
 
 		/* NO EREPORT(ERROR) from here till changes are logged */
@@ -310,33 +311,16 @@ FreeSpaceMapTruncateRel(Relation rel, BlockNumber nblocks)
 		UnlockReleaseBuffer(buf);
 
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address) + 1;
+		return new_nfsmblocks;
 	}
 	else
 	{
 		new_nfsmblocks = fsm_logical_to_physical(first_removed_address);
 		if (smgrnblocks(rel->rd_smgr, FSM_FORKNUM) <= new_nfsmblocks)
-			return;				/* nothing to do; the FSM was already smaller */
+			return InvalidBlockNumber;	/* nothing to do; the FSM was already smaller */
+		else
+			return new_nfsmblocks;
 	}
-
-	/* Truncate the unused FSM pages, and send smgr inval message */
-	smgrtruncate(rel->rd_smgr, FSM_FORKNUM, new_nfsmblocks);
-
-	/*
-	 * We might as well update the local smgr_fsm_nblocks setting.
-	 * smgrtruncate sent an smgr cache inval message, which will cause other
-	 * backends to invalidate their copy of smgr_fsm_nblocks, and this one too
-	 * at the next command boundary.  But this ensures it isn't outright wrong
-	 * until then.
-	 */
-	if (rel->rd_smgr)
-		rel->rd_smgr->smgr_fsm_nblocks = new_nfsmblocks;
-
-	/*
-	 * Update upper-level FSM pages to account for the truncation.  This is
-	 * important because the just-truncated pages were likely marked as
-	 * all-free, and would be preferentially selected.
-	 */
-	FreeSpaceMapVacuumRange(rel, nblocks, InvalidBlockNumber);
 }
 
 /*
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index 5b5a80e..b6d4d23 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -469,6 +469,7 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 	pfree(rnodes);
 }
 
+
 /*
  *	smgrextend() -- Add a new block to a file.
  *
@@ -561,15 +562,21 @@ smgrnblocks(SMgrRelation reln, ForkNumber forknum)
  *					  of blocks
  *
  * The truncation is done immediately, so this can't be rolled back.
+ *
+ * The caller must hold AccessExclusiveLock on the relation, to ensure that
+ * other backends receive the smgr invalidation event that this function sends
+ * before they access the relation again.
  */
 void
-smgrtruncate(SMgrRelation reln, ForkNumber forknum, BlockNumber nblocks)
+smgrtruncate(SMgrRelation reln, ForkNumber *forknum, int nforks, BlockNumber *nblocks)
 {
+	int		i;
+
 	/*
 	 * 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, nblocks);
+	DropRelFileNodeBuffers(reln->smgr_rnode, forknum, nforks, nblocks);
 
 	/*
 	 * Send a shared-inval message to force other backends to close any smgr
@@ -583,10 +590,23 @@ smgrtruncate(SMgrRelation reln, ForkNumber forknum, BlockNumber nblocks)
 	 */
 	CacheInvalidateSmgr(reln->smgr_rnode);
 
-	/*
-	 * Do the truncation.
-	 */
-	smgrsw[reln->smgr_which].smgr_truncate(reln, forknum, nblocks);
+	/* Do the truncation */
+	for (i = 0; i < nforks; i++)
+	{
+		smgrsw[reln->smgr_which].smgr_truncate(reln, forknum[i], nblocks[i]);
+
+		/*
+		 * We might as well update the local smgr_fsm_nblocks and smgr_vm_nblocks
+		 * setting. The smgr cache inval message we sent will cause other backends
+		 * to invalidate their copy of smgr_fsm_nblocks and smgr_vm_nblocks, and
+		 * these ones too at the next command boundary. But these ensure these
+		 * aren't outright wrong until then.
+		 */
+		if (forknum[i] == FSM_FORKNUM)
+			reln->smgr_fsm_nblocks = nblocks[i];
+		if (forknum[i] == VISIBILITYMAP_FORKNUM)
+			reln->smgr_vm_nblocks = nblocks[i];
+	}
 }
 
 /*
diff --git a/src/include/access/visibilitymap.h b/src/include/access/visibilitymap.h
index 2d88043..1ab6a81 100644
--- a/src/include/access/visibilitymap.h
+++ b/src/include/access/visibilitymap.h
@@ -44,6 +44,6 @@ extern void visibilitymap_set(Relation rel, BlockNumber heapBlk, Buffer heapBuf,
 							  uint8 flags);
 extern uint8 visibilitymap_get_status(Relation rel, BlockNumber heapBlk, Buffer *vmbuf);
 extern void visibilitymap_count(Relation rel, BlockNumber *all_visible, BlockNumber *all_frozen);
-extern void visibilitymap_truncate(Relation rel, BlockNumber nheapblocks);
+extern BlockNumber visibilitymap_truncate_prepare(Relation rel, BlockNumber nheapblocks);
 
 #endif							/* VISIBILITYMAP_H */
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 509f4b7..17b97f7 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -190,8 +190,8 @@ extern BlockNumber RelationGetNumberOfBlocksInFork(Relation relation,
 extern void FlushOneBuffer(Buffer buffer);
 extern void FlushRelationBuffers(Relation rel);
 extern void FlushDatabaseBuffers(Oid dbid);
-extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode,
-								   ForkNumber forkNum, BlockNumber firstDelBlock);
+extern void DropRelFileNodeBuffers(RelFileNodeBackend rnode, ForkNumber *forkNum,
+								   int nforks, BlockNumber *firstDelBlock);
 extern void DropRelFileNodesAllBuffers(RelFileNodeBackend *rnodes, int nnodes);
 extern void DropDatabaseBuffers(Oid dbid);
 
diff --git a/src/include/storage/freespace.h b/src/include/storage/freespace.h
index 8d8c465..0b834cb 100644
--- a/src/include/storage/freespace.h
+++ b/src/include/storage/freespace.h
@@ -30,7 +30,7 @@ extern void RecordPageWithFreeSpace(Relation rel, BlockNumber heapBlk,
 extern void XLogRecordPageWithFreeSpace(RelFileNode rnode, BlockNumber heapBlk,
 										Size spaceAvail);
 
-extern void FreeSpaceMapTruncateRel(Relation rel, BlockNumber nblocks);
+extern BlockNumber FreeSpaceMapLocateBlock(Relation rel, BlockNumber nblocks);
 extern void FreeSpaceMapVacuum(Relation rel);
 extern void FreeSpaceMapVacuumRange(Relation rel, BlockNumber start,
 									BlockNumber end);
diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h
index 7393727..1543d8d 100644
--- a/src/include/storage/smgr.h
+++ b/src/include/storage/smgr.h
@@ -101,8 +101,8 @@ 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 void smgrtruncate(SMgrRelation reln, ForkNumber forknum,
-						 BlockNumber nblocks);
+extern void smgrtruncate(SMgrRelation reln, ForkNumber *forknum,
+						 int nforks, BlockNumber *nblocks);
 extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum);
 extern void AtEOXact_SMgr(void);
 
