static bool
 heapam_scan_bitmap_next_block(TableScanDesc scan,
-                                                         TBMIterateResult *tbmres,
+                                                         BlockNumber *blockno, bool *recheck,
                                                          uint64 *lossy_pages, uint64 *exact_pages)
 {
        HeapScanDesc hscan = (HeapScanDesc) scan;
-       BlockNumber block = tbmres->blockno;
+       BlockNumber block;
        Buffer          buffer;
        Snapshot        snapshot;
        int                     ntup;
+       TBMIterateResult *tbmres;
 
        hscan->rs_cindex = 0;
        hscan->rs_ntuples = 0;
 
+       *blockno = InvalidBlockNumber;
+       *recheck = true;
+
+       do
+       {
+               CHECK_FOR_INTERRUPTS();
+
+               if (scan->st.bitmap.rs_shared_iterator)
+                       tbmres = tbm_shared_iterate(scan->st.bitmap.rs_shared_iterator);
+               else
+                       tbmres = tbm_iterate(scan->st.bitmap.rs_iterator);
+
+               if (tbmres == NULL)
+                       return false;
+
+               /*
+                * Ignore any claimed entries past what we think is the end of the
+                * relation. It may have been extended after the start of our scan (we
+                * only hold an AccessShareLock, and it could be inserts from this
+                * backend).  We don't take this optimization in SERIALIZABLE
+                * isolation though, as we need to examine all invisible tuples
+                * reachable by the index.
+                */
+       } while (!IsolationIsSerializable() &&
+                        tbmres->blockno >= hscan->rs_nblocks);
+
+       /* Got a valid block */
+       *blockno = tbmres->blockno;
+       *recheck = tbmres->recheck;
+
        /*
         * We can skip fetching the heap page if we don't need any fields from the
         * heap, the bitmap entries don't need rechecking, and all tuples on the
                return true;
        }
 
-       /*
-        * Ignore any claimed entries past what we think is the end of the
-        * relation. It may have been extended after the start of our scan (we
-        * only hold an AccessShareLock, and it could be inserts from this
-        * backend).  We don't take this optimization in SERIALIZABLE isolation
-        * though, as we need to examine all invisible tuples reachable by the
-        * index.
-        */
-       if (!IsolationIsSerializable() && block >= hscan->rs_nblocks)
-               return false;
+       block = tbmres->blockno;
 
        /*
         * Acquire pin on the target heap page, trading in any pin we held before.
        else
                (*lossy_pages)++;
 
-       return ntup > 0;
+       /*
+        * Return true to indicate that a valid block was found and the bitmap is
+        * not exhausted. If there are no visible tuples on this page,
+        * hscan->rs_ntuples will be 0 and heapam_scan_bitmap_next_tuple() will
+        * return false returning control to this function to advance to the next
+        * block in the bitmap.
+        */
+       return true;
 }
 
 static bool
 heapam_scan_bitmap_next_tuple(TableScanDesc scan,
-                                                         TBMIterateResult *tbmres,
                                                          TupleTableSlot *slot)
 {
        HeapScanDesc hscan = (HeapScanDesc) scan;
 
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static inline void BitmapDoneInitializingSharedState(ParallelBitmapHeapState *pstate);
-static inline void BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
-                                                                                               BlockNumber blockno);
+static inline void BitmapAdjustPrefetchIterator(BitmapHeapScanState *node);
 static inline void BitmapAdjustPrefetchTarget(BitmapHeapScanState *node);
 static inline void BitmapPrefetch(BitmapHeapScanState *node,
                                                                  TableScanDesc scan);
        ExprContext *econtext;
        TableScanDesc scan;
        TIDBitmap  *tbm;
-       TBMIterator *tbmiterator = NULL;
-       TBMSharedIterator *shared_tbmiterator = NULL;
-       TBMIterateResult *tbmres;
        TupleTableSlot *slot;
        ParallelBitmapHeapState *pstate = node->pstate;
        dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
        slot = node->ss.ss_ScanTupleSlot;
        scan = node->ss.ss_currentScanDesc;
        tbm = node->tbm;
-       if (pstate == NULL)
-               tbmiterator = node->tbmiterator;
-       else
-               shared_tbmiterator = node->shared_tbmiterator;
-       tbmres = node->tbmres;
 
        /*
         * If we haven't yet performed the underlying index scan, do it, and begin
         */
        if (!node->initialized)
        {
+               TBMIterator *tbmiterator = NULL;
+               TBMSharedIterator *shared_tbmiterator = NULL;
+
                if (!pstate)
                {
                        tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
                                elog(ERROR, "unrecognized result from subplan");
 
                        node->tbm = tbm;
-                       node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
-                       node->tbmres = tbmres = NULL;
+                       tbmiterator = tbm_begin_iterate(tbm);
 
 #ifdef USE_PREFETCH
                        if (node->prefetch_maximum > 0)
                        }
 
                        /* Allocate a private iterator and attach the shared state to it */
-                       node->shared_tbmiterator = shared_tbmiterator =
+                       shared_tbmiterator =
                                tbm_attach_shared_iterate(dsa, pstate->tbmiterator);
-                       node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
                        if (node->prefetch_maximum > 0)
                        node->ss.ss_currentScanDesc = scan;
                }
 
+               scan->st.bitmap.rs_iterator = tbmiterator;
+               scan->st.bitmap.rs_shared_iterator = shared_tbmiterator;
                node->initialized = true;
+
+               goto new_page;
        }
 
        for (;;)
        {
-               CHECK_FOR_INTERRUPTS();
-
-               /*
-                * Get next page of results if needed
-                */
-               if (tbmres == NULL)
-               {
-                       if (!pstate)
-                               node->tbmres = tbmres = tbm_iterate(tbmiterator);
-                       else
-                               node->tbmres = tbmres = tbm_shared_iterate(shared_tbmiterator);
-                       if (tbmres == NULL)
-                       {
-                               /* no more entries in the bitmap */
-                               break;
-                       }
-
-                       BitmapAdjustPrefetchIterator(node, tbmres->blockno);
-
-                       if (!table_scan_bitmap_next_block(scan, tbmres,
-                                                                                         &node->stats.lossy_pages,
-                                                                                         &node->stats.exact_pages))
-                       {
-                               /* AM doesn't think this block is valid, skip */
-                               continue;
-                       }
-
-                       /* Adjust the prefetch target */
-                       BitmapAdjustPrefetchTarget(node);
-               }
-               else
+               while (table_scan_bitmap_next_tuple(scan, slot))
                {
                        /*
                         * Continuing in previously obtained page.
                         */
 
+                       CHECK_FOR_INTERRUPTS();
+
 #ifdef USE_PREFETCH
 
                        /*
                                SpinLockRelease(&pstate->mutex);
                        }
 #endif                                                 /* USE_PREFETCH */
+
+                       /*
+                        * We issue prefetch requests *after* fetching the current page to
+                        * try to avoid having prefetching interfere with the main I/O.
+                        * Also, this should happen only when we have determined there is
+                        * still something to do on the current page, else we may
+                        * uselessly prefetch the same page we are just about to request
+                        * for real.
+                        */
+                       BitmapPrefetch(node, scan);
+
+                       /*
+                        * If we are using lossy info, we have to recheck the qual
+                        * conditions at every tuple.
+                        */
+                       if (node->recheck)
+                       {
+                               econtext->ecxt_scantuple = slot;
+                               if (!ExecQualAndReset(node->bitmapqualorig, econtext))
+                               {
+                                       /* Fails recheck, so drop it and loop back for another */
+                                       InstrCountFiltered2(node, 1);
+                                       ExecClearTuple(slot);
+                                       continue;
+                               }
+                       }
+
+                       /* OK to return this tuple */
+                       return slot;
                }
 
-               /*
-                * We issue prefetch requests *after* fetching the current page to try
-                * to avoid having prefetching interfere with the main I/O. Also, this
-                * should happen only when we have determined there is still something
-                * to do on the current page, else we may uselessly prefetch the same
-                * page we are just about to request for real.
-                */
-               BitmapPrefetch(node, scan);
+new_page:
+
+               BitmapAdjustPrefetchIterator(node);
 
                /*
-                * Attempt to fetch tuple from AM.
+                * Returns false if the bitmap is exhausted and there are no further
+                * blocks we need to scan.
                 */
-               if (!table_scan_bitmap_next_tuple(scan, tbmres, slot))
-               {
-                       /* nothing more to look at on this page */
-                       node->tbmres = tbmres = NULL;
-                       continue;
-               }
+               if (!table_scan_bitmap_next_block(scan, &node->blockno,
+                                                                                 &node->recheck,
+                                                                                 &node->stats.lossy_pages,
+                                                                                 &node->stats.exact_pages))
+                       break;
 
                /*
-                * If we are using lossy info, we have to recheck the qual conditions
-                * at every tuple.
+                * If serial, we can error out if the the prefetch block doesn't stay
+                * ahead of the current block.
                 */
-               if (tbmres->recheck)
-               {
-                       econtext->ecxt_scantuple = slot;
-                       if (!ExecQualAndReset(node->bitmapqualorig, econtext))
-                       {
-                               /* Fails recheck, so drop it and loop back for another */
-                               InstrCountFiltered2(node, 1);
-                               ExecClearTuple(slot);
-                               continue;
-                       }
-               }
-
-               /* OK to return this tuple */
-               return slot;
+               if (node->pstate == NULL &&
+                       node->prefetch_iterator &&
+                       node->prefetch_blockno < node->blockno)
+                       elog(ERROR,
+                                "prefetch and main iterators are out of sync. pfblockno: %d. blockno: %d",
+                                node->prefetch_blockno, node->blockno);
+
+               /* Adjust the prefetch target */
+               BitmapAdjustPrefetchTarget(node);
        }
 
        /*
 
 /*
  *     BitmapAdjustPrefetchIterator - Adjust the prefetch iterator
+ *
+ *     We keep track of how far the prefetch iterator is ahead of the main
+ *     iterator in prefetch_pages. For each block the main iterator returns, we
+ *     decrement prefetch_pages.
  */
 static inline void
-BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
-                                                        BlockNumber blockno)
+BitmapAdjustPrefetchIterator(BitmapHeapScanState *node)
 {
 #ifdef USE_PREFETCH
        ParallelBitmapHeapState *pstate = node->pstate;
+       TBMIterateResult *tbmpre;
 
        if (pstate == NULL)
        {
                }
                else if (prefetch_iterator)
                {
-                       /* Do not let the prefetch iterator get behind the main one */
-                       TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
-
-                       if (tbmpre == NULL || tbmpre->blockno != blockno)
-                               elog(ERROR, "prefetch and main iterators are out of sync");
+                       tbmpre = tbm_iterate(prefetch_iterator);
+                       node->prefetch_blockno = tbmpre ? tbmpre->blockno :
+                               InvalidBlockNumber;
                }
                return;
        }
 
+       /*
+        * XXX: There is a known issue with keeping the prefetch and current block
+        * iterators in sync for parallel bitmap table scans. This can lead to
+        * prefetching blocks that have already been read. See the discussion
+        * here:
+        * https://postgr.es/m/20240315211449.en2jcmdqxv5o6tlz%40alap3.anarazel.de
+        * Note that moving the call site of BitmapAdjustPrefetchIterator()
+        * exacerbates the effects of this bug.
+        */
        if (node->prefetch_maximum > 0)
        {
                TBMSharedIterator *prefetch_iterator = node->shared_prefetch_iterator;
                         * case.
                         */
                        if (prefetch_iterator)
-                               tbm_shared_iterate(prefetch_iterator);
+                       {
+                               tbmpre = tbm_shared_iterate(prefetch_iterator);
+                               node->prefetch_blockno = tbmpre ? tbmpre->blockno :
+                                       InvalidBlockNumber;
+                       }
                }
        }
 #endif                                                 /* USE_PREFETCH */
                                        break;
                                }
                                node->prefetch_pages++;
+                               node->prefetch_blockno = tbmpre->blockno;
 
                                /*
                                 * If we expect not to have to actually read this heap page,
                                        break;
                                }
 
+                               node->prefetch_blockno = tbmpre->blockno;
+
                                /* As above, skip prefetch if we expect not to need page */
                                skip_fetch = (!(scan->rs_flags & SO_NEED_TUPLES) &&
                                                          !tbmpre->recheck &&
 {
        PlanState  *outerPlan = outerPlanState(node);
 
-       /* rescan to release any page pin */
-       if (node->ss.ss_currentScanDesc)
+       TableScanDesc scan = node->ss.ss_currentScanDesc;
+
+       if (scan)
+       {
+               /*
+                * End iteration on iterators saved in scan descriptor.
+                */
+               if (scan->st.bitmap.rs_shared_iterator)
+               {
+                       tbm_end_shared_iterate(scan->st.bitmap.rs_shared_iterator);
+                       scan->st.bitmap.rs_shared_iterator = NULL;
+               }
+
+               if (scan->st.bitmap.rs_iterator)
+               {
+                       tbm_end_iterate(scan->st.bitmap.rs_iterator);
+                       scan->st.bitmap.rs_iterator = NULL;
+               }
+
+               /* rescan to release any page pin */
                table_rescan(node->ss.ss_currentScanDesc, NULL);
+       }
 
        /* release bitmaps and buffers if any */
-       if (node->tbmiterator)
-               tbm_end_iterate(node->tbmiterator);
        if (node->prefetch_iterator)
                tbm_end_iterate(node->prefetch_iterator);
-       if (node->shared_tbmiterator)
-               tbm_end_shared_iterate(node->shared_tbmiterator);
        if (node->shared_prefetch_iterator)
                tbm_end_shared_iterate(node->shared_prefetch_iterator);
        if (node->tbm)
        if (node->pvmbuffer != InvalidBuffer)
                ReleaseBuffer(node->pvmbuffer);
        node->tbm = NULL;
-       node->tbmiterator = NULL;
-       node->tbmres = NULL;
        node->prefetch_iterator = NULL;
        node->initialized = false;
-       node->shared_tbmiterator = NULL;
        node->shared_prefetch_iterator = NULL;
        node->pvmbuffer = InvalidBuffer;
+       node->recheck = true;
+       node->blockno = InvalidBlockNumber;
+       node->prefetch_blockno = InvalidBlockNumber;
 
        ExecScanReScan(&node->ss);
 
         */
        ExecEndNode(outerPlanState(node));
 
+       if (scanDesc)
+       {
+               /*
+                * End iteration on iterators saved in scan descriptor.
+                */
+               if (scanDesc->st.bitmap.rs_shared_iterator)
+               {
+                       tbm_end_shared_iterate(scanDesc->st.bitmap.rs_shared_iterator);
+                       scanDesc->st.bitmap.rs_shared_iterator = NULL;
+               }
+
+               if (scanDesc->st.bitmap.rs_iterator)
+               {
+                       tbm_end_iterate(scanDesc->st.bitmap.rs_iterator);
+                       scanDesc->st.bitmap.rs_iterator = NULL;
+               }
+
+               /*
+                * close table scan
+                */
+               table_endscan(scanDesc);
+       }
+
        /*
         * release bitmaps and buffers if any
         */
-       if (node->tbmiterator)
-               tbm_end_iterate(node->tbmiterator);
        if (node->prefetch_iterator)
                tbm_end_iterate(node->prefetch_iterator);
        if (node->tbm)
                tbm_free(node->tbm);
-       if (node->shared_tbmiterator)
-               tbm_end_shared_iterate(node->shared_tbmiterator);
        if (node->shared_prefetch_iterator)
                tbm_end_shared_iterate(node->shared_prefetch_iterator);
        if (node->pvmbuffer != InvalidBuffer)
                ReleaseBuffer(node->pvmbuffer);
-
-       /*
-        * close heap scan
-        */
-       if (scanDesc)
-               table_endscan(scanDesc);
-
 }
 
 /* ----------------------------------------------------------------
        scanstate->ss.ps.ExecProcNode = ExecBitmapHeapScan;
 
        scanstate->tbm = NULL;
-       scanstate->tbmiterator = NULL;
-       scanstate->tbmres = NULL;
        scanstate->pvmbuffer = InvalidBuffer;
 
        /* Zero the statistics counters */
        scanstate->prefetch_pages = 0;
        scanstate->prefetch_target = 0;
        scanstate->initialized = false;
-       scanstate->shared_tbmiterator = NULL;
        scanstate->shared_prefetch_iterator = NULL;
        scanstate->pstate = NULL;
+       scanstate->recheck = true;
+       scanstate->blockno = InvalidBlockNumber;
+       scanstate->prefetch_blockno = InvalidBlockNumber;
 
        /*
         * Miscellaneous initialization
 
 struct BulkInsertStateData;
 struct IndexInfo;
 struct SampleScanState;
-struct TBMIterateResult;
 struct VacuumParams;
 struct ValidateIndexState;
 
         */
 
        /*
-        * Prepare to fetch / check / return tuples from `tbmres->blockno` as part
-        * of a bitmap table scan. `scan` was started via table_beginscan_bm().
-        * Return false if there are no tuples to be found on the page, true
-        * otherwise.
+        * Prepare to fetch / check / return tuples from `blockno` as part of a
+        * bitmap table scan. `scan` was started via table_beginscan_bm(). Return
+        * false if the bitmap is exhausted and true otherwise.
         *
         * This will typically read and pin the target block, and do the necessary
         * work to allow scan_bitmap_next_tuple() to return tuples (e.g. it might
-        * make sense to perform tuple visibility checks at this time). For some
-        * AMs it will make more sense to do all the work referencing `tbmres`
-        * contents here, for others it might be better to defer more work to
-        * scan_bitmap_next_tuple.
-        *
-        * If `tbmres->blockno` is -1, this is a lossy scan and all visible tuples
-        * on the page have to be returned, otherwise the tuples at offsets in
-        * `tbmres->offsets` need to be returned.
+        * make sense to perform tuple visibility checks at this time).
         *
         * `lossy_pages` and `exact_pages` are EXPLAIN counters that can be
         * incremented by the table AM to indicate whether or not the block's
         * representation in the bitmap is lossy.
         *
+        * `recheck` is set by the table AM to indicate whether or not the tuples
+        * from this block should be rechecked. Tuples from lossy pages will
+        * always need to be rechecked, but some non-lossy pages' tuples may also
+        * require recheck.
+        *
+        * `blockno` is the current block and is set by the table AM. The table AM
+        * is responsible for advancing the main iterator, but the bitmap table
+        * scan code still advances the prefetch iterator. `blockno` is used by
+        * bitmap table scan code to validate that the prefetch block stays ahead
+        * of the current block.
+        *
         * XXX: Currently this may only be implemented if the AM uses md.c as its
         * storage manager, and uses ItemPointer->ip_blkid in a manner that maps
         * blockids directly to the underlying storage. nodeBitmapHeapscan.c
         * scan_bitmap_next_tuple need to exist, or neither.
         */
        bool            (*scan_bitmap_next_block) (TableScanDesc scan,
-                                                                                  struct TBMIterateResult *tbmres,
+                                                                                  BlockNumber *blockno,
+                                                                                  bool *recheck,
                                                                                   uint64 *lossy_pages,
                                                                                   uint64 *exact_pages);
 
         * Fetch the next tuple of a bitmap table scan into `slot` and return true
         * if a visible tuple was found, false otherwise.
         *
-        * For some AMs it will make more sense to do all the work referencing
-        * `tbmres` contents in scan_bitmap_next_block, for others it might be
-        * better to defer more work to this callback.
-        *
         * Optional callback, but either both scan_bitmap_next_block and
         * scan_bitmap_next_tuple need to exist, or neither.
         */
        bool            (*scan_bitmap_next_tuple) (TableScanDesc scan,
-                                                                                  struct TBMIterateResult *tbmres,
                                                                                   TupleTableSlot *slot);
 
        /*
 table_beginscan_bm(Relation rel, Snapshot snapshot,
                                   int nkeys, struct ScanKeyData *key, bool need_tuple)
 {
+       TableScanDesc result;
        uint32          flags = SO_TYPE_BITMAPSCAN | SO_ALLOW_PAGEMODE;
 
        if (need_tuple)
                flags |= SO_NEED_TUPLES;
 
-       return rel->rd_tableam->scan_begin(rel, snapshot, nkeys, key, NULL, flags);
+       result = rel->rd_tableam->scan_begin(rel, snapshot, nkeys, key,
+                                                                                NULL, flags);
+       result->st.bitmap.rs_shared_iterator = NULL;
+       result->st.bitmap.rs_iterator = NULL;
+       return result;
 }
 
 /*
  */
 
 /*
- * Prepare to fetch / check / return tuples from `tbmres->blockno` as part of
- * a bitmap table scan. `scan` needs to have been started via
- * table_beginscan_bm(). Returns false if there are no tuples to be found on
- * the page, true otherwise.
+ * Prepare to fetch / check / return tuples as part of a bitmap table scan.
+ * `scan` needs to have been started via table_beginscan_bm(). Returns false
+ * if there are no more blocks in the bitmap, true otherwise.
  *
  * `lossy_pages` and `exact_pages` are EXPLAIN counters that can be
  * incremented by the table AM to indicate whether or not the block's
  * representation in the bitmap is lossy.
  *
+ * `recheck` is set by the table AM to indicate whether or not the tuples
+ * from this block should be rechecked.
+ *
+ * `blockno` is the current block and is set by the table AM and is used by
+ * bitmap table scan code to validate that the prefetch block stays ahead of
+ * the current block.
+ *
  * Note, this is an optionally implemented function, therefore should only be
  * used after verifying the presence (at plan time or such).
  */
 static inline bool
 table_scan_bitmap_next_block(TableScanDesc scan,
-                                                        struct TBMIterateResult *tbmres,
+                                                        BlockNumber *blockno,
+                                                        bool *recheck,
                                                         uint64 *lossy_pages,
                                                         uint64 *exact_pages)
 {
                elog(ERROR, "unexpected table_scan_bitmap_next_block call during logical decoding");
 
        return scan->rs_rd->rd_tableam->scan_bitmap_next_block(scan,
-                                                                                                                  tbmres,
+                                                                                                                  blockno, recheck,
                                                                                                                   lossy_pages,
                                                                                                                   exact_pages);
 }
  */
 static inline bool
 table_scan_bitmap_next_tuple(TableScanDesc scan,
-                                                        struct TBMIterateResult *tbmres,
                                                         TupleTableSlot *slot)
 {
        /*
                elog(ERROR, "unexpected table_scan_bitmap_next_tuple call during logical decoding");
 
        return scan->rs_rd->rd_tableam->scan_bitmap_next_tuple(scan,
-                                                                                                                  tbmres,
                                                                                                                   slot);
 }