return scan->rs_prefetch_block;
 }
 
+/*
+ * Read stream API callback for bitmap heap scans.
+ * Returns the next block the caller wants from the read stream or
+ * InvalidBlockNumber when done.
+ */
+static BlockNumber
+bitmapheap_stream_read_next(ReadStream *pgsr, void *private_data,
+                           void *per_buffer_data)
+{
+   TBMIterateResult *tbmres = per_buffer_data;
+   BitmapHeapScanDesc bscan = (BitmapHeapScanDesc) private_data;
+   HeapScanDesc hscan = (HeapScanDesc) bscan;
+   TableScanDesc sscan = &hscan->rs_base;
+
+   for (;;)
+   {
+       CHECK_FOR_INTERRUPTS();
+
+       /* no more entries in the bitmap */
+       if (!tbm_iterate(&sscan->st.rs_tbmiterator, tbmres))
+           return InvalidBlockNumber;
+
+       /*
+        * 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() &&
+           tbmres->blockno >= hscan->rs_nblocks)
+           continue;
+
+       /*
+        * 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 page are visible to our transaction.
+        */
+       if (!(sscan->rs_flags & SO_NEED_TUPLES) &&
+           !tbmres->recheck &&
+           VM_ALL_VISIBLE(sscan->rs_rd, tbmres->blockno, &bscan->rs_vmbuffer))
+       {
+           OffsetNumber offsets[TBM_MAX_TUPLES_PER_PAGE];
+           int         noffsets;
+
+           /* can't be lossy in the skip_fetch case */
+           Assert(!tbmres->lossy);
+           Assert(bscan->rs_empty_tuples_pending >= 0);
+
+           /*
+            * We throw away the offsets, but this is the easiest way to get a
+            * count of tuples.
+            */
+           noffsets = tbm_extract_page_tuple(tbmres, offsets, TBM_MAX_TUPLES_PER_PAGE);
+           bscan->rs_empty_tuples_pending += noffsets;
+           continue;
+       }
+
+       return tbmres->blockno;
+   }
+
+   /* not reachable */
+   Assert(false);
+}
+
 /* ----------------
  *     initscan - scan code common to heap_beginscan and heap_rescan
  * ----------------
    scan->rs_base.rs_flags = flags;
    scan->rs_base.rs_parallel = parallel_scan;
    scan->rs_strategy = NULL;   /* set in initscan */
+   scan->rs_cbuf = InvalidBuffer;
 
    /*
     * Disable page-at-a-time mode if it's not a MVCC-safe snapshot.
                                                          scan,
                                                          0);
    }
+   else if (scan->rs_base.rs_flags & SO_TYPE_BITMAPSCAN)
+   {
+       scan->rs_read_stream = read_stream_begin_relation(READ_STREAM_DEFAULT,
+                                                         scan->rs_strategy,
+                                                         scan->rs_base.rs_rd,
+                                                         MAIN_FORKNUM,
+                                                         bitmapheap_stream_read_next,
+                                                         scan,
+                                                         sizeof(TBMIterateResult));
+   }
 
 
    return (TableScanDesc) scan;
     * unpin scan buffers
     */
    if (BufferIsValid(scan->rs_cbuf))
+   {
        ReleaseBuffer(scan->rs_cbuf);
+       scan->rs_cbuf = InvalidBuffer;
+   }
 
    if (scan->rs_base.rs_flags & SO_TYPE_BITMAPSCAN)
    {
 
 
 static bool
 heapam_scan_bitmap_next_block(TableScanDesc scan,
-                             BlockNumber *blockno, bool *recheck,
+                             bool *recheck,
                              uint64 *lossy_pages, uint64 *exact_pages)
 {
    BitmapHeapScanDesc bscan = (BitmapHeapScanDesc) scan;
    HeapScanDesc hscan = (HeapScanDesc) bscan;
    BlockNumber block;
+   void       *per_buffer_data;
    Buffer      buffer;
    Snapshot    snapshot;
    int         ntup;
-   TBMIterateResult tbmres;
+   TBMIterateResult *tbmres;
    OffsetNumber offsets[TBM_MAX_TUPLES_PER_PAGE];
    int         noffsets = -1;
 
    Assert(scan->rs_flags & SO_TYPE_BITMAPSCAN);
+   Assert(hscan->rs_read_stream);
 
    hscan->rs_cindex = 0;
    hscan->rs_ntuples = 0;
 
-   *blockno = InvalidBlockNumber;
-   *recheck = true;
-
-   do
+   /* Release buffer containing previous block. */
+   if (BufferIsValid(hscan->rs_cbuf))
    {
-       CHECK_FOR_INTERRUPTS();
+       ReleaseBuffer(hscan->rs_cbuf);
+       hscan->rs_cbuf = InvalidBuffer;
+   }
 
-       if (!tbm_iterate(&scan->st.rs_tbmiterator, &tbmres))
-           return false;
+   hscan->rs_cbuf = read_stream_next_buffer(hscan->rs_read_stream,
+                                            &per_buffer_data);
 
-       /* Exact pages need their tuple offsets extracted. */
-       if (!tbmres.lossy)
-           noffsets = tbm_extract_page_tuple(&tbmres, offsets,
-                                             TBM_MAX_TUPLES_PER_PAGE);
+   if (BufferIsInvalid(hscan->rs_cbuf))
+   {
+       if (BufferIsValid(bscan->rs_vmbuffer))
+       {
+           ReleaseBuffer(bscan->rs_vmbuffer);
+           bscan->rs_vmbuffer = InvalidBuffer;
+       }
 
        /*
-        * 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.
+        * Bitmap is exhausted. Time to emit empty tuples if relevant. We emit
+        * all empty tuples at the end instead of emitting them per block we
+        * skip fetching. This is necessary because the streaming read API
+        * will only return TBMIterateResults for blocks actually fetched.
+        * When we skip fetching a block, we keep track of how many empty
+        * tuples to emit at the end of the BitmapHeapScan. We do not recheck
+        * all NULL tuples.
         */
-   } while (!IsolationIsSerializable() &&
-            tbmres.blockno >= hscan->rs_nblocks);
+       *recheck = false;
+       return bscan->rs_empty_tuples_pending > 0;
+   }
 
-   /* Got a valid block */
-   *blockno = tbmres.blockno;
-   *recheck = tbmres.recheck;
+   Assert(per_buffer_data);
 
-   /*
-    * 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
-    * page are visible to our transaction.
-    */
-   if (!(scan->rs_flags & SO_NEED_TUPLES) &&
-       !tbmres.recheck &&
-       VM_ALL_VISIBLE(scan->rs_rd, tbmres.blockno, &bscan->rs_vmbuffer))
-   {
-       /* can't be lossy in the skip_fetch case */
-       Assert(!tbmres.lossy);
-       Assert(bscan->rs_empty_tuples_pending >= 0);
-       Assert(noffsets > -1);
+   tbmres = per_buffer_data;
 
-       bscan->rs_empty_tuples_pending += noffsets;
+   Assert(BlockNumberIsValid(tbmres->blockno));
+   Assert(BufferGetBlockNumber(hscan->rs_cbuf) == tbmres->blockno);
 
-       return true;
-   }
+   /* Exact pages need their tuple offsets extracted. */
+   if (!tbmres->lossy)
+       noffsets = tbm_extract_page_tuple(tbmres, offsets,
+                                         TBM_MAX_TUPLES_PER_PAGE);
 
-   block = tbmres.blockno;
+   *recheck = tbmres->recheck;
 
-   /*
-    * Acquire pin on the target heap page, trading in any pin we held before.
-    */
-   hscan->rs_cbuf = ReleaseAndReadBuffer(hscan->rs_cbuf,
-                                         scan->rs_rd,
-                                         block);
-   hscan->rs_cblock = block;
+   block = hscan->rs_cblock = tbmres->blockno;
    buffer = hscan->rs_cbuf;
    snapshot = scan->rs_snapshot;
 
    /*
     * We need two separate strategies for lossy and non-lossy cases.
     */
-   if (!tbmres.lossy)
+   if (!tbmres->lossy)
    {
        /*
         * Bitmap is non-lossy, so we just look through the offsets listed in
    Assert(ntup <= MaxHeapTuplesPerPage);
    hscan->rs_ntuples = ntup;
 
-   if (tbmres.lossy)
+   if (tbmres->lossy)
        (*lossy_pages)++;
    else
        (*exact_pages)++;
 
 static void BitmapTableScanSetup(BitmapHeapScanState *node);
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static inline void BitmapDoneInitializingSharedState(ParallelBitmapHeapState *pstate);
-static inline void BitmapAdjustPrefetchIterator(BitmapHeapScanState *node);
-static inline void BitmapAdjustPrefetchTarget(BitmapHeapScanState *node);
-static inline void BitmapPrefetch(BitmapHeapScanState *node,
-                                 TableScanDesc scan);
 static bool BitmapShouldInitializeSharedState(ParallelBitmapHeapState *pstate);
 
 
  * Do the underlying index scan, build the bitmap, set up the parallel state
  * needed for parallel workers to iterate through the bitmap, and set up the
  * underlying table scan descriptor.
- *
- * For prefetching, we use *two* iterators, one for the pages we are actually
- * scanning and another that runs ahead of the first for prefetching.
- * node->prefetch_pages tracks exactly how many pages ahead the prefetch
- * iterator is.  Also, node->prefetch_target tracks the desired prefetch
- * distance, which starts small and increases up to the
- * node->prefetch_maximum.  This is to avoid doing a lot of prefetching in a
- * scan that stops after a few tuples because of a LIMIT.
  */
 static void
 BitmapTableScanSetup(BitmapHeapScanState *node)
         */
        pstate->tbmiterator = tbm_prepare_shared_iterate(node->tbm);
 
-#ifdef USE_PREFETCH
-       if (node->prefetch_maximum > 0)
-       {
-           pstate->prefetch_iterator =
-               tbm_prepare_shared_iterate(node->tbm);
-       }
-#endif                         /* USE_PREFETCH */
-
        /* We have initialized the shared state so wake up others. */
        BitmapDoneInitializingSharedState(pstate);
    }
                                    pstate->tbmiterator :
                                    InvalidDsaPointer);
 
-#ifdef USE_PREFETCH
-   if (node->prefetch_maximum > 0)
-       node->prefetch_iterator =
-           tbm_begin_iterate(node->tbm, dsa,
-                             pstate ?
-                             pstate->prefetch_iterator :
-                             InvalidDsaPointer);
-#endif                         /* USE_PREFETCH */
-
    /*
     * If this is the first scan of the underlying table, create the table
     * scan descriptor and begin the scan.
    node->initialized = true;
 }
 
-
 /* ----------------------------------------------------------------
  *     BitmapHeapNext
  *
    TableScanDesc scan;
    TupleTableSlot *slot;
 
-#ifdef USE_PREFETCH
-   ParallelBitmapHeapState *pstate = node->pstate;
-#endif
-
    /*
     * extract necessary information from index scan node
     */
 
            CHECK_FOR_INTERRUPTS();
 
-#ifdef USE_PREFETCH
-
-           /*
-            * Try to prefetch at least a few pages even before we get to the
-            * second page if we don't stop reading after the first tuple.
-            */
-           if (!pstate)
-           {
-               if (node->prefetch_target < node->prefetch_maximum)
-                   node->prefetch_target++;
-           }
-           else if (pstate->prefetch_target < node->prefetch_maximum)
-           {
-               /* take spinlock while updating shared state */
-               SpinLockAcquire(&pstate->mutex);
-               if (pstate->prefetch_target < node->prefetch_maximum)
-                   pstate->prefetch_target++;
-               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.
 
 new_page:
 
-       BitmapAdjustPrefetchIterator(node);
-
        /*
         * Returns false if the bitmap is exhausted and there are no further
         * blocks we need to scan.
         */
-       if (!table_scan_bitmap_next_block(scan, &node->blockno,
+       if (!table_scan_bitmap_next_block(scan,
                                          &node->recheck,
                                          &node->stats.lossy_pages,
                                          &node->stats.exact_pages))
            break;
-
-       /*
-        * If serial, we can error out if the prefetch block doesn't stay
-        * ahead of the current block.
-        */
-       if (node->pstate == NULL &&
-           !tbm_exhausted(&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);
    }
 
    /*
    ConditionVariableBroadcast(&pstate->cv);
 }
 
-/*
- * 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)
-{
-#ifdef USE_PREFETCH
-   ParallelBitmapHeapState *pstate = node->pstate;
-   TBMIterateResult tbmpre;
-
-   if (pstate == NULL)
-   {
-       TBMIterator *prefetch_iterator = &node->prefetch_iterator;
-
-       if (node->prefetch_pages > 0)
-       {
-           /* The main iterator has closed the distance by one page */
-           node->prefetch_pages--;
-       }
-       else if (!tbm_exhausted(prefetch_iterator))
-       {
-           tbm_iterate(prefetch_iterator, &tbmpre);
-           node->prefetch_blockno = tbmpre.blockno;
-       }
-       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)
-   {
-       TBMIterator *prefetch_iterator = &node->prefetch_iterator;
-
-       SpinLockAcquire(&pstate->mutex);
-       if (pstate->prefetch_pages > 0)
-       {
-           pstate->prefetch_pages--;
-           SpinLockRelease(&pstate->mutex);
-       }
-       else
-       {
-           /* Release the mutex before iterating */
-           SpinLockRelease(&pstate->mutex);
-
-           /*
-            * In case of shared mode, we can not ensure that the current
-            * blockno of the main iterator and that of the prefetch iterator
-            * are same.  It's possible that whatever blockno we are
-            * prefetching will be processed by another process.  Therefore,
-            * we don't validate the blockno here as we do in non-parallel
-            * case.
-            */
-           if (!tbm_exhausted(prefetch_iterator))
-           {
-               tbm_iterate(prefetch_iterator, &tbmpre);
-               node->prefetch_blockno = tbmpre.blockno;
-           }
-       }
-   }
-#endif                         /* USE_PREFETCH */
-}
-
-/*
- * BitmapAdjustPrefetchTarget - Adjust the prefetch target
- *
- * Increase prefetch target if it's not yet at the max.  Note that
- * we will increase it to zero after fetching the very first
- * page/tuple, then to one after the second tuple is fetched, then
- * it doubles as later pages are fetched.
- */
-static inline void
-BitmapAdjustPrefetchTarget(BitmapHeapScanState *node)
-{
-#ifdef USE_PREFETCH
-   ParallelBitmapHeapState *pstate = node->pstate;
-
-   if (pstate == NULL)
-   {
-       if (node->prefetch_target >= node->prefetch_maximum)
-            /* don't increase any further */ ;
-       else if (node->prefetch_target >= node->prefetch_maximum / 2)
-           node->prefetch_target = node->prefetch_maximum;
-       else if (node->prefetch_target > 0)
-           node->prefetch_target *= 2;
-       else
-           node->prefetch_target++;
-       return;
-   }
-
-   /* Do an unlocked check first to save spinlock acquisitions. */
-   if (pstate->prefetch_target < node->prefetch_maximum)
-   {
-       SpinLockAcquire(&pstate->mutex);
-       if (pstate->prefetch_target >= node->prefetch_maximum)
-            /* don't increase any further */ ;
-       else if (pstate->prefetch_target >= node->prefetch_maximum / 2)
-           pstate->prefetch_target = node->prefetch_maximum;
-       else if (pstate->prefetch_target > 0)
-           pstate->prefetch_target *= 2;
-       else
-           pstate->prefetch_target++;
-       SpinLockRelease(&pstate->mutex);
-   }
-#endif                         /* USE_PREFETCH */
-}
-
-/*
- * BitmapPrefetch - Prefetch, if prefetch_pages are behind prefetch_target
- */
-static inline void
-BitmapPrefetch(BitmapHeapScanState *node, TableScanDesc scan)
-{
-#ifdef USE_PREFETCH
-   ParallelBitmapHeapState *pstate = node->pstate;
-
-   if (pstate == NULL)
-   {
-       TBMIterator *prefetch_iterator = &node->prefetch_iterator;
-
-       if (!tbm_exhausted(prefetch_iterator))
-       {
-           while (node->prefetch_pages < node->prefetch_target)
-           {
-               TBMIterateResult tbmpre;
-               bool        skip_fetch;
-
-               if (!tbm_iterate(prefetch_iterator, &tbmpre))
-               {
-                   /* No more pages to prefetch */
-                   Assert(!BlockNumberIsValid(tbmpre.blockno));
-                   tbm_end_iterate(prefetch_iterator);
-                   break;
-               }
-               node->prefetch_pages++;
-               node->prefetch_blockno = tbmpre.blockno;
-
-               /*
-                * If we expect not to have to actually read this heap page,
-                * skip this prefetch call, but continue to run the prefetch
-                * logic normally.  (Would it be better not to increment
-                * prefetch_pages?)
-                */
-               skip_fetch = (!(scan->rs_flags & SO_NEED_TUPLES) &&
-                             !tbmpre.recheck &&
-                             VM_ALL_VISIBLE(node->ss.ss_currentRelation,
-                                            tbmpre.blockno,
-                                            &node->pvmbuffer));
-
-               if (!skip_fetch)
-                   PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre.blockno);
-           }
-       }
-
-       return;
-   }
-
-   if (pstate->prefetch_pages < pstate->prefetch_target)
-   {
-       TBMIterator *prefetch_iterator = &node->prefetch_iterator;
-
-       if (!tbm_exhausted(prefetch_iterator))
-       {
-           while (1)
-           {
-               TBMIterateResult tbmpre;
-               bool        do_prefetch = false;
-               bool        skip_fetch;
-
-               /*
-                * Recheck under the mutex. If some other process has already
-                * done enough prefetching then we need not to do anything.
-                */
-               SpinLockAcquire(&pstate->mutex);
-               if (pstate->prefetch_pages < pstate->prefetch_target)
-               {
-                   pstate->prefetch_pages++;
-                   do_prefetch = true;
-               }
-               SpinLockRelease(&pstate->mutex);
-
-               if (!do_prefetch)
-                   return;
-
-               if (!tbm_iterate(prefetch_iterator, &tbmpre))
-               {
-                   Assert(!BlockNumberIsValid(tbmpre.blockno));
-                   /* No more pages to prefetch */
-                   tbm_end_iterate(prefetch_iterator);
-                   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 &&
-                             VM_ALL_VISIBLE(node->ss.ss_currentRelation,
-                                            tbmpre.blockno,
-                                            &node->pvmbuffer));
-
-               if (!skip_fetch)
-                   PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre.blockno);
-           }
-       }
-   }
-#endif                         /* USE_PREFETCH */
-}
-
 /*
  * BitmapHeapRecheck -- access method routine to recheck a tuple in EvalPlanQual
  */
        table_rescan(node->ss.ss_currentScanDesc, NULL);
    }
 
-   /* If we did not already clean up the prefetch iterator, do so now. */
-   if (!tbm_exhausted(&node->prefetch_iterator))
-       tbm_end_iterate(&node->prefetch_iterator);
-
    /* release bitmaps and buffers if any */
    if (node->tbm)
        tbm_free(node->tbm);
-   if (node->pvmbuffer != InvalidBuffer)
-       ReleaseBuffer(node->pvmbuffer);
    node->tbm = NULL;
    node->initialized = false;
-   node->pvmbuffer = InvalidBuffer;
    node->recheck = true;
-   /* Only used for serial BHS */
-   node->blockno = InvalidBlockNumber;
-   node->prefetch_blockno = InvalidBlockNumber;
-   node->prefetch_pages = 0;
-   node->prefetch_target = -1;
 
    ExecScanReScan(&node->ss);
 
        table_endscan(scanDesc);
    }
 
-   /* If we did not already clean up the prefetch iterator, do so now. */
-   if (!tbm_exhausted(&node->prefetch_iterator))
-       tbm_end_iterate(&node->prefetch_iterator);
-
    /*
     * release bitmaps and buffers if any
     */
    if (node->tbm)
        tbm_free(node->tbm);
-   if (node->pvmbuffer != InvalidBuffer)
-       ReleaseBuffer(node->pvmbuffer);
 }
 
 /* ----------------------------------------------------------------
    scanstate->ss.ps.ExecProcNode = ExecBitmapHeapScan;
 
    scanstate->tbm = NULL;
-   scanstate->pvmbuffer = InvalidBuffer;
 
    /* Zero the statistics counters */
    memset(&scanstate->stats, 0, sizeof(BitmapHeapScanInstrumentation));
 
-   scanstate->prefetch_pages = 0;
-   scanstate->prefetch_target = -1;
    scanstate->initialized = false;
    scanstate->pstate = NULL;
    scanstate->recheck = true;
-   scanstate->blockno = InvalidBlockNumber;
-   scanstate->prefetch_blockno = InvalidBlockNumber;
 
    /*
     * Miscellaneous initialization
    scanstate->bitmapqualorig =
        ExecInitQual(node->bitmapqualorig, (PlanState *) scanstate);
 
-   /*
-    * Maximum number of prefetches for the tablespace if configured,
-    * otherwise the current value of the effective_io_concurrency GUC.
-    */
-   scanstate->prefetch_maximum =
-       get_tablespace_io_concurrency(currentRelation->rd_rel->reltablespace);
-
    scanstate->ss.ss_currentRelation = currentRelation;
 
    /*
        sinstrument = (SharedBitmapHeapInstrumentation *) ptr;
 
    pstate->tbmiterator = 0;
-   pstate->prefetch_iterator = 0;
 
    /* Initialize the mutex */
    SpinLockInit(&pstate->mutex);
-   pstate->prefetch_pages = 0;
-   pstate->prefetch_target = -1;
    pstate->state = BM_INITIAL;
 
    ConditionVariableInit(&pstate->cv);
        return;
 
    pstate->state = BM_INITIAL;
-   pstate->prefetch_pages = 0;
-   pstate->prefetch_target = -1;
 
    if (DsaPointerIsValid(pstate->tbmiterator))
        tbm_free_shared_area(dsa, pstate->tbmiterator);
 
-   if (DsaPointerIsValid(pstate->prefetch_iterator))
-       tbm_free_shared_area(dsa, pstate->prefetch_iterator);
-
    pstate->tbmiterator = InvalidDsaPointer;
-   pstate->prefetch_iterator = InvalidDsaPointer;
 }
 
 /* ----------------------------------------------------------------