BitmapHeapScan uses the read stream API
authorMelanie Plageman <melanieplageman@gmail.com>
Sat, 15 Mar 2025 14:34:42 +0000 (10:34 -0400)
committerMelanie Plageman <melanieplageman@gmail.com>
Sat, 15 Mar 2025 14:34:42 +0000 (10:34 -0400)
Make Bitmap Heap Scan use the read stream API instead of invoking
ReadBuffer() for each block indicated by the bitmap.

The read stream API handles prefetching, so remove all of the explicit
prefetching from bitmap heap scan code.

Now, heap table AM implements a read stream callback which uses the
bitmap iterator to return the next required block to the read stream
code.

Tomas Vondra conducted extensive regression testing of this feature.
Andres Freund, Thomas Munro, and I analyzed regressions and Thomas Munro
patched the read stream API.

Author: Melanie Plageman <melanieplageman@gmail.com>
Reviewed-by: Tomas Vondra <tomas@vondra.me>
Tested-by: Tomas Vondra <tomas@vondra.me>
Tested-by: Andres Freund <andres@anarazel.de>
Tested-by: Thomas Munro <thomas.munro@gmail.com>
Tested-by: Nazir Bilal Yavuz <byavuz81@gmail.com>
Discussion: https://postgr.es/m/flat/CAAKRu_ZwCwWFeL_H3ia26bP2e7HiKLWt0ZmGXPVwPO6uXq0vaA%40mail.gmail.com

src/backend/access/heap/heapam.c
src/backend/access/heap/heapam_handler.c
src/backend/executor/nodeBitmapHeapscan.c
src/include/access/tableam.h
src/include/nodes/execnodes.h

index 21575a8ffef0f5a79260675aec829c544223229d..b12b583c4d9004718dda5cbb6211eb17f89155c1 100644 (file)
@@ -280,6 +280,72 @@ heap_scan_stream_read_next_serial(ReadStream *stream,
    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
  * ----------------
@@ -1068,6 +1134,7 @@ heap_beginscan(Relation relation, Snapshot snapshot,
    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.
@@ -1147,6 +1214,16 @@ heap_beginscan(Relation relation, Snapshot 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;
@@ -1181,7 +1258,10 @@ heap_rescan(TableScanDesc sscan, ScanKey key, bool set_params,
     * 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)
    {
index 25d26409e2cd619a8292505fb257ca83ea351322..3035adacade279426c4e716ddb4df43dc2a0785e 100644 (file)
@@ -2117,82 +2117,72 @@ heapam_estimate_rel_size(Relation rel, int32 *attr_widths,
 
 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;
 
@@ -2213,7 +2203,7 @@ heapam_scan_bitmap_next_block(TableScanDesc scan,
    /*
     * 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
@@ -2277,7 +2267,7 @@ heapam_scan_bitmap_next_block(TableScanDesc scan,
    Assert(ntup <= MaxHeapTuplesPerPage);
    hscan->rs_ntuples = ntup;
 
-   if (tbmres.lossy)
+   if (tbmres->lossy)
        (*lossy_pages)++;
    else
        (*exact_pages)++;
index 3b4ea0f61446e8f9f09bfaab153a89a4ce7d264e..6df34094a1303dc6e22e4b8d0f1ce963c78b2544 100644 (file)
 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);
 
 
@@ -62,14 +58,6 @@ 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)
@@ -102,14 +90,6 @@ 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);
    }
@@ -119,15 +99,6 @@ BitmapTableScanSetup(BitmapHeapScanState *node)
                                    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.
@@ -158,7 +129,6 @@ BitmapTableScanSetup(BitmapHeapScanState *node)
    node->initialized = true;
 }
 
-
 /* ----------------------------------------------------------------
  *     BitmapHeapNext
  *
@@ -172,10 +142,6 @@ BitmapHeapNext(BitmapHeapScanState *node)
    TableScanDesc scan;
    TupleTableSlot *slot;
 
-#ifdef USE_PREFETCH
-   ParallelBitmapHeapState *pstate = node->pstate;
-#endif
-
    /*
     * extract necessary information from index scan node
     */
@@ -204,37 +170,6 @@ BitmapHeapNext(BitmapHeapScanState *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.
@@ -257,31 +192,15 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 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);
    }
 
    /*
@@ -305,225 +224,6 @@ BitmapDoneInitializingSharedState(ParallelBitmapHeapState *pstate)
    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
  */
@@ -580,24 +280,12 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
        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);
 
@@ -666,17 +354,11 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node)
        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);
 }
 
 /* ----------------------------------------------------------------
@@ -709,18 +391,13 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
    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
@@ -760,13 +437,6 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
    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;
 
    /*
@@ -870,12 +540,9 @@ ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
        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);
@@ -912,17 +579,11 @@ ExecBitmapHeapReInitializeDSM(BitmapHeapScanState *node,
        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;
 }
 
 /* ----------------------------------------------------------------
index 131c050c15f139b9288baa578f0c238d4f2736fe..507d4ebe68f81bc7b9e77865a863c07f4829987e 100644 (file)
@@ -797,28 +797,12 @@ typedef struct TableAmRoutine
     * 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
-    * performs prefetching directly using that interface.  This probably
-    * needs to be rectified at a later point.
-    *
-    * XXX: Currently this may only be implemented if the AM uses the
-    * visibilitymap, as nodeBitmapHeapscan.c unconditionally accesses it to
-    * perform prefetching.  This probably needs to be rectified at a later
-    * point.
+    * Prefetching additional data from the bitmap is left to the table AM.
     *
     * Optional callback, but either both scan_bitmap_next_block and
     * scan_bitmap_next_tuple need to exist, or neither.
     */
    bool        (*scan_bitmap_next_block) (TableScanDesc scan,
-                                          BlockNumber *blockno,
                                           bool *recheck,
                                           uint64 *lossy_pages,
                                           uint64 *exact_pages);
@@ -1966,16 +1950,11 @@ table_relation_estimate_size(Relation rel, int32 *attr_widths,
  * `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,
-                            BlockNumber *blockno,
                             bool *recheck,
                             uint64 *lossy_pages,
                             uint64 *exact_pages)
@@ -1989,7 +1968,7 @@ table_scan_bitmap_next_block(TableScanDesc scan,
        elog(ERROR, "unexpected table_scan_bitmap_next_block call during logical decoding");
 
    return scan->rs_rd->rd_tableam->scan_bitmap_next_block(scan,
-                                                          blockno, recheck,
+                                                          recheck,
                                                           lossy_pages,
                                                           exact_pages);
 }
index 575b0b1bd246a86e5c078dada0a49e56bb226655..d4d4e655180cf50ff842cd48b502c668433257b4 100644 (file)
@@ -1840,11 +1840,7 @@ typedef enum
 /* ----------------
  *  ParallelBitmapHeapState information
  *     tbmiterator             iterator for scanning current pages
- *     prefetch_iterator       iterator for prefetching ahead of current page
- *     mutex                   mutual exclusion for the prefetching variable
- *                             and state
- *     prefetch_pages          # pages prefetch iterator is ahead of current
- *     prefetch_target         current target prefetch distance
+ *     mutex                   mutual exclusion for state
  *     state                   current state of the TIDBitmap
  *     cv                      conditional wait variable
  * ----------------
@@ -1852,10 +1848,7 @@ typedef enum
 typedef struct ParallelBitmapHeapState
 {
    dsa_pointer tbmiterator;
-   dsa_pointer prefetch_iterator;
    slock_t     mutex;
-   int         prefetch_pages;
-   int         prefetch_target;
    SharedBitmapState state;
    ConditionVariable cv;
 } ParallelBitmapHeapState;
@@ -1879,18 +1872,11 @@ typedef struct SharedBitmapHeapInstrumentation
  *
  *     bitmapqualorig     execution state for bitmapqualorig expressions
  *     tbm                bitmap obtained from child index scan(s)
- *     pvmbuffer          buffer for visibility-map lookups of prefetched pages
  *     stats              execution statistics
- *     prefetch_iterator  iterator for prefetching ahead of current page
- *     prefetch_pages     # pages prefetch iterator is ahead of current
- *     prefetch_target    current target prefetch distance
- *     prefetch_maximum   maximum value for prefetch_target
  *     initialized        is node is ready to iterate
  *     pstate             shared state for parallel bitmap scan
  *     sinstrument        statistics for parallel workers
  *     recheck            do current page's tuples need recheck
- *     blockno            used to validate pf and current block stay in sync
- *     prefetch_blockno   used to validate pf stays ahead of current block
  * ----------------
  */
 typedef struct BitmapHeapScanState
@@ -1898,18 +1884,11 @@ typedef struct BitmapHeapScanState
    ScanState   ss;             /* its first field is NodeTag */
    ExprState  *bitmapqualorig;
    TIDBitmap  *tbm;
-   Buffer      pvmbuffer;
    BitmapHeapScanInstrumentation stats;
-   TBMIterator prefetch_iterator;
-   int         prefetch_pages;
-   int         prefetch_target;
-   int         prefetch_maximum;
    bool        initialized;
    ParallelBitmapHeapState *pstate;
    SharedBitmapHeapInstrumentation *sinstrument;
    bool        recheck;
-   BlockNumber blockno;
-   BlockNumber prefetch_blockno;
 } BitmapHeapScanState;
 
 /* ----------------