From 2976fc013eb79cea3e72ac129145ba086111f25b Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Fri, 15 Aug 2025 11:01:52 -0400 Subject: [PATCH 1/3] bufmgr: aio: Prototype for not waiting for already-in-progress IO Author: Reviewed-by: Discussion: https://postgr.es/m/zljergweqti7x67lg5ije2rzjusie37nslsnkjkkby4laqqbfw@3p3zu522yykv Backpatch: --- src/backend/storage/buffer/bufmgr.c | 150 ++++++++++++++++++++++++++-- src/include/storage/bufmgr.h | 1 + 2 files changed, 142 insertions(+), 9 deletions(-) diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c index 00719c7aea2e..f602767a0d31 100644 --- a/src/backend/storage/buffer/bufmgr.c +++ b/src/backend/storage/buffer/bufmgr.c @@ -1538,6 +1538,46 @@ ReadBuffersCanStartIOOnce(Buffer buffer, bool nowait) return StartBufferIO(GetBufferDescriptor(buffer - 1), true, nowait); } +/* + * Check if the buffer is already undergoing read AIO. If it is, assign the + * IO's wait reference to operation->io_wref, thereby allowing the caller to + * wait for that IO. + */ +static inline bool +ReadBuffersIOAlreadyInProgress(ReadBuffersOperation *operation, Buffer buffer) +{ + BufferDesc *desc; + uint32 buf_state; + PgAioWaitRef iow; + + pgaio_wref_clear(&iow); + + if (BufferIsLocal(buffer)) + { + desc = GetLocalBufferDescriptor(-buffer - 1); + buf_state = pg_atomic_read_u32(&desc->state); + if ((buf_state & BM_IO_IN_PROGRESS) && !(buf_state & BM_VALID)) + iow = desc->io_wref; + } + else + { + desc = GetBufferDescriptor(buffer - 1); + buf_state = LockBufHdr(desc); + + if ((buf_state & BM_IO_IN_PROGRESS) && !(buf_state & BM_VALID)) + iow = desc->io_wref; + UnlockBufHdr(desc, buf_state); + } + + if (pgaio_wref_valid(&iow)) + { + operation->io_wref = iow; + return true; + } + + return false; +} + /* * Helper for AsyncReadBuffers that tries to get the buffer ready for IO. */ @@ -1670,7 +1710,7 @@ WaitReadBuffers(ReadBuffersOperation *operation) * * we first check if we already know the IO is complete. */ - if (aio_ret->result.status == PGAIO_RS_UNKNOWN && + if ((operation->foreign_io || aio_ret->result.status == PGAIO_RS_UNKNOWN) && !pgaio_wref_check_done(&operation->io_wref)) { instr_time io_start = pgstat_prepare_io_time(track_io_timing); @@ -1689,11 +1729,66 @@ WaitReadBuffers(ReadBuffersOperation *operation) Assert(pgaio_wref_check_done(&operation->io_wref)); } - /* - * We now are sure the IO completed. Check the results. This - * includes reporting on errors if there were any. - */ - ProcessReadBuffersResult(operation); + if (unlikely(operation->foreign_io)) + { + Buffer buffer = operation->buffers[operation->nblocks_done]; + BufferDesc *desc; + uint32 buf_state; + + if (BufferIsLocal(buffer)) + { + desc = GetLocalBufferDescriptor(-buffer - 1); + buf_state = pg_atomic_read_u32(&desc->state); + } + else + { + desc = GetBufferDescriptor(buffer - 1); + buf_state = LockBufHdr(desc); + UnlockBufHdr(desc, buf_state); + } + + if (buf_state & BM_VALID) + { + operation->nblocks_done += 1; + Assert(operation->nblocks_done <= operation->nblocks); + + /* + * Report and track this as a 'hit' for this backend, even + * though it must have started out as a miss in + * PinBufferForBlock(). The other backend (or ourselves, + * as part of a read started earlier) will track this as a + * 'read'. + */ + TRACE_POSTGRESQL_BUFFER_READ_DONE(operation->forknum, + operation->blocknum + operation->nblocks_done, + operation->smgr->smgr_rlocator.locator.spcOid, + operation->smgr->smgr_rlocator.locator.dbOid, + operation->smgr->smgr_rlocator.locator.relNumber, + operation->smgr->smgr_rlocator.backend, + true); + + if (BufferIsLocal(buffer)) + pgBufferUsage.local_blks_hit += 1; + else + pgBufferUsage.shared_blks_hit += 1; + + if (operation->rel) + pgstat_count_buffer_hit(operation->rel); + + pgstat_count_io_op(io_object, io_context, IOOP_HIT, 1, 0); + + if (VacuumCostActive) + VacuumCostBalance += VacuumCostPageHit; + } + } + else + { + /* + * We now are sure the IO completed. Check the results. This + * includes reporting on errors if there were any. + */ + ProcessReadBuffersResult(operation); + } } /* @@ -1779,6 +1874,43 @@ AsyncReadBuffers(ReadBuffersOperation *operation, int *nblocks_progress) io_object = IOOBJECT_RELATION; } + /* + * If AIO is in progress, be it in this backend or another backend, we + * just associate the wait reference with the operation and wait in + * WaitReadBuffers(). This turns out to be important for performance in + * two workloads: + * + * 1) A read stream that has to read the same block multiple times within + * the readahead distance. This can happen e.g. for the table accesses of + * an index scan. + * + * 2) Concurrent scans by multiple backends on the same relation. + * + * If we were to synchronously wait for the in-progress IO, we'd not be + * able to keep enough I/O in flight. + * + * If we do find there is ongoing I/O for the buffer, we set up a 1-block + * ReadBuffersOperation that WaitReadBuffers then can wait on. + * + * It's possible that another backend starts IO on the buffer between this + * check and the ReadBuffersCanStartIO(nowait = false) below. In that case + * we will synchronously wait for the IO below, but the window for that is + * small enough that it won't happen often enough to have a significant + * performance impact. + */ + if (ReadBuffersIOAlreadyInProgress(operation, buffers[nblocks_done])) + { + *nblocks_progress = 1; + operation->foreign_io = true; + + CheckReadBuffersOperation(operation, false); + + + return true; + } + + operation->foreign_io = false; + /* * If zero_damaged_pages is enabled, add the READ_BUFFERS_ZERO_ON_ERROR * flag. The reason for that is that, hopefully, zero_damaged_pages isn't @@ -1836,9 +1968,9 @@ AsyncReadBuffers(ReadBuffersOperation *operation, int *nblocks_progress) /* * Check if we can start IO on the first to-be-read buffer. * - * If an I/O is already in progress in another backend, we want to wait - * for the outcome: either done, or something went wrong and we will - * retry. + * If a synchronous I/O is in progress in another backend (it can't be + * this backend), we want to wait for the outcome: either done, or + * something went wrong and we will retry. */ if (!ReadBuffersCanStartIO(buffers[nblocks_done], false)) { diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h index b5f8f3c5d42f..139499111878 100644 --- a/src/include/storage/bufmgr.h +++ b/src/include/storage/bufmgr.h @@ -147,6 +147,7 @@ struct ReadBuffersOperation int flags; int16 nblocks; int16 nblocks_done; + bool foreign_io; PgAioWaitRef io_wref; PgAioReturn io_return; }; From 52812290b113ddfd8a333a77ae45d11a15196057 Mon Sep 17 00:00:00 2001 From: Peter Geoghegan Date: Tue, 9 Sep 2025 19:50:03 -0400 Subject: [PATCH 2/3] Add amgetbatch interface for index scan prefetching. Allows the index AM to provide items (TIDs and tuples) in batches, which is then used to implement prefetching of heap tuples in index scans (including index-only scans). This is similar to prefetching already done in bitmap scans, and can result in significant speedups. The index AM may implement an optional "amgetbatch" callback, returning a batch of items. The indexam.c code then handles this transparently through the existing "getnext" interface. --- contrib/bloom/blutils.c | 3 +- doc/src/sgml/indexam.sgml | 105 +- src/backend/access/brin/brin.c | 5 +- src/backend/access/gin/ginutil.c | 5 +- src/backend/access/gist/gist.c | 5 +- src/backend/access/hash/hash.c | 5 +- src/backend/access/heap/heapam_handler.c | 43 +- src/backend/access/index/genam.c | 1 + src/backend/access/index/indexam.c | 1325 ++++++++++++++++- src/backend/access/nbtree/nbtree.c | 311 ++-- src/backend/access/nbtree/nbtsearch.c | 712 ++++----- src/backend/access/nbtree/nbtutils.c | 70 +- src/backend/access/spgist/spgutils.c | 5 +- src/backend/commands/indexcmds.c | 2 +- src/backend/executor/execAmi.c | 2 +- src/backend/executor/nodeIndexonlyscan.c | 101 +- src/backend/optimizer/path/costsize.c | 1 + src/backend/optimizer/util/plancat.c | 6 +- src/backend/replication/logical/relation.c | 3 +- src/backend/storage/aio/read_stream.c | 14 +- src/backend/utils/adt/amutils.c | 4 +- src/backend/utils/adt/selfuncs.c | 4 + src/backend/utils/misc/guc_parameters.dat | 7 + src/backend/utils/misc/postgresql.conf.sample | 1 + src/include/access/amapi.h | 22 +- src/include/access/genam.h | 4 + src/include/access/heapam.h | 1 + src/include/access/nbtree.h | 100 +- src/include/access/relscan.h | 150 ++ src/include/nodes/pathnodes.h | 2 +- src/include/optimizer/cost.h | 1 + .../modules/dummy_index_am/dummy_index_am.c | 3 +- src/test/regress/expected/sysviews.out | 3 +- src/tools/pgindent/typedefs.list | 8 +- 34 files changed, 2175 insertions(+), 859 deletions(-) diff --git a/contrib/bloom/blutils.c b/contrib/bloom/blutils.c index 2c0e71eedc65..3e00149304f5 100644 --- a/contrib/bloom/blutils.c +++ b/contrib/bloom/blutils.c @@ -148,8 +148,7 @@ blhandler(PG_FUNCTION_ARGS) amroutine->amgettuple = NULL; amroutine->amgetbitmap = blgetbitmap; amroutine->amendscan = blendscan; - amroutine->ammarkpos = NULL; - amroutine->amrestrpos = NULL; + amroutine->amposreset = NULL; amroutine->amestimateparallelscan = NULL; amroutine->aminitparallelscan = NULL; amroutine->amparallelrescan = NULL; diff --git a/doc/src/sgml/indexam.sgml b/doc/src/sgml/indexam.sgml index 63d7e376f195..4e8225332c4f 100644 --- a/doc/src/sgml/indexam.sgml +++ b/doc/src/sgml/indexam.sgml @@ -161,10 +161,11 @@ typedef struct IndexAmRoutine ambeginscan_function ambeginscan; amrescan_function amrescan; amgettuple_function amgettuple; /* can be NULL */ + amgetbatch_function amgetbatch; /* can be NULL */ + amfreebatch_function amfreebatch; /* can be NULL */ amgetbitmap_function amgetbitmap; /* can be NULL */ amendscan_function amendscan; - ammarkpos_function ammarkpos; /* can be NULL */ - amrestrpos_function amrestrpos; /* can be NULL */ + amposreset_function amposreset; /* can be NULL */ /* interface functions to support parallel index scans */ amestimateparallelscan_function amestimateparallelscan; /* can be NULL */ @@ -746,6 +747,63 @@ amgettuple (IndexScanDesc scan, +IndexScanBatch +amgetbatch (IndexScanDesc scan, + IndexScanBatch batch, + ScanDirection direction); + + Return the next batch of index tuples in the given scan, moving in the given + direction (forward or backward in the index). Returns an instance of + IndexScanBatch with index tuples loaded, or + NULL if there are no more index tuples. The caller may + provide a pre-allocated IndexScanBatch instance, + in which case the index tuples are loaded into it instead of allocating + a new one. The same caveats described for amgettuple + apply here too. When an entry in the returned batch means only that the + index contains an entry that matches the scan keys, not that the tuple + necessarily still exists in the heap or will pass the caller's snapshot + test. + + + + The amgetbatch function need only be provided if the access + method supports plain index scans. If it doesn't, the + amgetbatch field in its IndexAmRoutine + struct must be set to NULL. + + + + The access method may provide only one of amgettuple and + amgetbatch callbacks, not both. When the access method + provides amgetbatch, it has to also povide + amfreebatch. + + + + The IndexScanBatch returned by amgetbatch + is no longer managed by the access method. It is up to the caller to decide + when it should be reused or freed by passing it to amfreebatch. + + + + +bool +amfreebatch (IndexScanDesc scan, + IndexScanBatch batch); + + Releases the batch returned by the amgetbatch earlier. + This frees all AM-specific resources associated with the batch, like buffer + pins, allocated memory, etc. + + + + The amfreebatch function need only be provided if the access + method provides amgetbatch. Otherwise it has to remain + set to NULL. + + + + int64 amgetbitmap (IndexScanDesc scan, TIDBitmap *tbm); @@ -789,32 +847,25 @@ amendscan (IndexScanDesc scan); void -ammarkpos (IndexScanDesc scan); +amposreset (IndexScanDesc scan); - Mark current scan position. The access method need only support one - remembered scan position per scan. - - - - The ammarkpos function need only be provided if the access - method supports ordered scans. If it doesn't, - the ammarkpos field in its IndexAmRoutine - struct may be set to NULL. - - - - -void -amrestrpos (IndexScanDesc scan); - - Restore the scan to the most recently marked position. - - - - The amrestrpos function need only be provided if the access - method supports ordered scans. If it doesn't, - the amrestrpos field in its IndexAmRoutine - struct may be set to NULL. + Notify index AM that core code will change the scan's position to an item + returned as part of an earlier batch. The index AM must therefore + invalidate any state that independently tracks the scan's progress + (e.g., array keys used with a ScalarArrayOpExpr qual). Called by the core + system when it is about to restore a mark. + + + + The amposreset function need only be provided if the access + method supports ordered scans through the amgetbatch + interface. If it doesn't, the amposreset field + in its IndexAmRoutine struct should be set to + NULL. Index AMs that don't have any private state that might need to be + invalidated might still find it useful to provide an empty + amposreset function; if amposreset + is set to NULL, the core system will assume that it is unsafe to restore a + marked position. diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c index 2f7d1437919d..7a9c8711eecd 100644 --- a/src/backend/access/brin/brin.c +++ b/src/backend/access/brin/brin.c @@ -294,10 +294,11 @@ brinhandler(PG_FUNCTION_ARGS) amroutine->ambeginscan = brinbeginscan; amroutine->amrescan = brinrescan; amroutine->amgettuple = NULL; + amroutine->amgetbatch = NULL; + amroutine->amfreebatch = NULL; amroutine->amgetbitmap = bringetbitmap; amroutine->amendscan = brinendscan; - amroutine->ammarkpos = NULL; - amroutine->amrestrpos = NULL; + amroutine->amposreset = NULL; amroutine->amestimateparallelscan = NULL; amroutine->aminitparallelscan = NULL; amroutine->amparallelrescan = NULL; diff --git a/src/backend/access/gin/ginutil.c b/src/backend/access/gin/ginutil.c index 78f7b7a2495c..9b55ab230945 100644 --- a/src/backend/access/gin/ginutil.c +++ b/src/backend/access/gin/ginutil.c @@ -82,10 +82,11 @@ ginhandler(PG_FUNCTION_ARGS) amroutine->ambeginscan = ginbeginscan; amroutine->amrescan = ginrescan; amroutine->amgettuple = NULL; + amroutine->amgetbatch = NULL; + amroutine->amfreebatch = NULL; amroutine->amgetbitmap = gingetbitmap; amroutine->amendscan = ginendscan; - amroutine->ammarkpos = NULL; - amroutine->amrestrpos = NULL; + amroutine->amposreset = NULL; amroutine->amestimateparallelscan = NULL; amroutine->aminitparallelscan = NULL; amroutine->amparallelrescan = NULL; diff --git a/src/backend/access/gist/gist.c b/src/backend/access/gist/gist.c index 5213cd71e977..06600f47a9a3 100644 --- a/src/backend/access/gist/gist.c +++ b/src/backend/access/gist/gist.c @@ -103,10 +103,11 @@ gisthandler(PG_FUNCTION_ARGS) amroutine->ambeginscan = gistbeginscan; amroutine->amrescan = gistrescan; amroutine->amgettuple = gistgettuple; + amroutine->amgetbatch = NULL; + amroutine->amfreebatch = NULL; amroutine->amgetbitmap = gistgetbitmap; amroutine->amendscan = gistendscan; - amroutine->ammarkpos = NULL; - amroutine->amrestrpos = NULL; + amroutine->amposreset = NULL; amroutine->amestimateparallelscan = NULL; amroutine->aminitparallelscan = NULL; amroutine->amparallelrescan = NULL; diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c index 53061c819fbf..b4b4607241e5 100644 --- a/src/backend/access/hash/hash.c +++ b/src/backend/access/hash/hash.c @@ -102,10 +102,11 @@ hashhandler(PG_FUNCTION_ARGS) amroutine->ambeginscan = hashbeginscan; amroutine->amrescan = hashrescan; amroutine->amgettuple = hashgettuple; + amroutine->amgetbatch = NULL; + amroutine->amfreebatch = NULL; amroutine->amgetbitmap = hashgetbitmap; amroutine->amendscan = hashendscan; - amroutine->ammarkpos = NULL; - amroutine->amrestrpos = NULL; + amroutine->amposreset = NULL; amroutine->amestimateparallelscan = NULL; amroutine->aminitparallelscan = NULL; amroutine->amparallelrescan = NULL; diff --git a/src/backend/access/heap/heapam_handler.c b/src/backend/access/heap/heapam_handler.c index bcbac844bb66..6c41b3119ea4 100644 --- a/src/backend/access/heap/heapam_handler.c +++ b/src/backend/access/heap/heapam_handler.c @@ -84,7 +84,9 @@ heapam_index_fetch_begin(Relation rel) IndexFetchHeapData *hscan = palloc0(sizeof(IndexFetchHeapData)); hscan->xs_base.rel = rel; + hscan->xs_base.rs = NULL; hscan->xs_cbuf = InvalidBuffer; + hscan->xs_blk = InvalidBlockNumber; return &hscan->xs_base; } @@ -94,10 +96,14 @@ heapam_index_fetch_reset(IndexFetchTableData *scan) { IndexFetchHeapData *hscan = (IndexFetchHeapData *) scan; + if (scan->rs) + read_stream_reset(scan->rs); + if (BufferIsValid(hscan->xs_cbuf)) { ReleaseBuffer(hscan->xs_cbuf); hscan->xs_cbuf = InvalidBuffer; + hscan->xs_blk = InvalidBlockNumber; } } @@ -108,6 +114,9 @@ heapam_index_fetch_end(IndexFetchTableData *scan) heapam_index_fetch_reset(scan); + if (scan->rs) + read_stream_end(scan->rs); + pfree(hscan); } @@ -124,23 +133,37 @@ heapam_index_fetch_tuple(struct IndexFetchTableData *scan, Assert(TTS_IS_BUFFERTUPLE(slot)); - /* We can skip the buffer-switching logic if we're in mid-HOT chain. */ - if (!*call_again) + /* + * Switch to correct buffer if we don't have it already (we can skip this + * if we're in mid-HOT chain) + */ + if (!*call_again && hscan->xs_blk != ItemPointerGetBlockNumber(tid)) { - /* Switch to correct buffer if we don't have it already */ - Buffer prev_buf = hscan->xs_cbuf; + /* Remember this buffer's block number for next time */ + hscan->xs_blk = ItemPointerGetBlockNumber(tid); - hscan->xs_cbuf = ReleaseAndReadBuffer(hscan->xs_cbuf, - hscan->xs_base.rel, - ItemPointerGetBlockNumber(tid)); + if (BufferIsValid(hscan->xs_cbuf)) + ReleaseBuffer(hscan->xs_cbuf); /* - * Prune page, but only if we weren't already on this page + * When using a read stream, the stream will already know which block + * number comes next (though an assertion will verify a match below) */ - if (prev_buf != hscan->xs_cbuf) - heap_page_prune_opt(hscan->xs_base.rel, hscan->xs_cbuf); + if (scan->rs) + hscan->xs_cbuf = read_stream_next_buffer(scan->rs, NULL); + else + hscan->xs_cbuf = ReadBuffer(hscan->xs_base.rel, hscan->xs_blk); + + /* + * Prune page when it is pinned for the first time + */ + heap_page_prune_opt(hscan->xs_base.rel, hscan->xs_cbuf); } + /* Assert that the TID's block number's buffer is now pinned */ + Assert(BufferIsValid(hscan->xs_cbuf)); + Assert(BufferGetBlockNumber(hscan->xs_cbuf) == hscan->xs_blk); + /* Obtain share-lock on the buffer so we can examine visibility */ LockBuffer(hscan->xs_cbuf, BUFFER_LOCK_SHARE); got_heap_tuple = heap_hot_search_buffer(tid, diff --git a/src/backend/access/index/genam.c b/src/backend/access/index/genam.c index 0cb27af13109..55e60c9ffde2 100644 --- a/src/backend/access/index/genam.c +++ b/src/backend/access/index/genam.c @@ -89,6 +89,7 @@ RelationGetIndexScan(Relation indexRelation, int nkeys, int norderbys) scan->xs_snapshot = InvalidSnapshot; /* caller must initialize this */ scan->numberOfKeys = nkeys; scan->numberOfOrderBys = norderbys; + scan->batchState = NULL; /* used by amgetbatch index AMs */ /* * We allocate key workspace here, but it won't get filled until amrescan. diff --git a/src/backend/access/index/indexam.c b/src/backend/access/index/indexam.c index 0492d92d23b1..fd3ffa222bec 100644 --- a/src/backend/access/index/indexam.c +++ b/src/backend/access/index/indexam.c @@ -44,6 +44,7 @@ #include "postgres.h" #include "access/amapi.h" +#include "access/nbtree.h" /* XXX for MaxTIDsPerBTreePage (should remove) */ #include "access/relation.h" #include "access/reloptions.h" #include "access/relscan.h" @@ -51,9 +52,11 @@ #include "catalog/index.h" #include "catalog/pg_type.h" #include "nodes/execnodes.h" +#include "optimizer/cost.h" #include "pgstat.h" #include "storage/lmgr.h" #include "storage/predicate.h" +#include "utils/memdebug.h" #include "utils/ruleutils.h" #include "utils/snapmgr.h" #include "utils/syscache.h" @@ -107,8 +110,146 @@ do { \ static IndexScanDesc index_beginscan_internal(Relation indexRelation, int nkeys, int norderbys, Snapshot snapshot, ParallelIndexScanDesc pscan, bool temp_snap); +static ItemPointer index_batch_getnext_tid(IndexScanDesc scan, ScanDirection direction); +static ItemPointer index_retail_getnext_tid(IndexScanDesc scan, ScanDirection direction); static inline void validate_relation_kind(Relation r); +/* index batching */ +static void index_batch_init(IndexScanDesc scan); +static void index_batch_reset(IndexScanDesc scan, bool complete); +static void index_batch_end(IndexScanDesc scan); +static bool index_batch_getnext(IndexScanDesc scan, ScanDirection direction); +static void index_batch_free(IndexScanDesc scan, IndexScanBatch batch); + +static BlockNumber index_scan_stream_read_next(ReadStream *stream, + void *callback_private_data, + void *per_buffer_data); + +static pg_attribute_always_inline bool index_batch_pos_advance(IndexScanDesc scan, + IndexScanBatchPos *pos, + ScanDirection direction); +static void index_batch_pos_reset(IndexScanDesc scan, IndexScanBatchPos *pos); +static void index_batch_kill_item(IndexScanDesc scan); + +static void AssertCheckBatchPosValid(IndexScanDesc scan, IndexScanBatchPos *pos); +static void AssertCheckBatch(IndexScanDesc scan, IndexScanBatch batch); +static void AssertCheckBatches(IndexScanDesc scan); + + +/* + * Maximum number of batches (leaf pages) we can keep in memory. + * + * The value 64 value is arbitrary, it's about 1MB of data with 8KB pages + * (512kB for pages, and then a bit of overhead). We should not really need + * this many batches in most cases, though. The read stream looks ahead just + * enough to queue enough IOs, adjusting the distance (TIDs, but ultimately + * the number of future batches) to meet that. + * + * In most cases an index leaf page has many (hundreds) index tuples, and + * it's enough to read one or maybe two leaf pages ahead to satisfy the + * distance. + * + * But there are cases where this may not quite work, for example: + * + * a) bloated index - many pages only have a single index item, so that + * achieving the distance requires too many leaf pages + * + * b) correlated index - duplicate blocks are skipped (the callback does not + * even return those, thanks to currentPrefetchBlock optimization), and are + * mostly ignored in the distance heuristics (read stream does not even see + * those TIDs, and there's no I/O either) + * + * c) index-only scan - the callback skips TIDs from all-visible blocks (not + * reading those is the whole point of index-only scans), and so it's + * invisible to the distance / IO heuristics (similarly to duplicates) + * + * In these cases we might need to read a significant number of batches to + * find the first block to return to the read stream. It's not clear if + * looking this far ahead is worth it - it's a lot of work / synchronous + * I/O, and the query may terminate before reaching those TIDs (e.g. due to + * a LIMIT clause). + * + * Currently, there's no way to "pause" a read stream - stop looking ahead + * for a while, but then resume the work when a batch gets freed. To simulate + * this, the read stream is terminated (as if there were no more data), and + * then reset after draining all the queued blocks in order to resume work. + * This works, but it "stalls" the I/O queue. If it happens very often, it + * can be a serious performance bottleneck. + + * XXX Maybe 64 is too high? It also defines the maximum amount of overhead + * allowed. In the worst case, reading a single row might trigger reading this + * many leaf pages (e.g. with IOS, if most pages are all-visible). Which might + * be an issue with LIMIT queries, when we actually won't get that far. + */ +#define INDEX_SCAN_MAX_BATCHES 64 + +/* + * Thresholds controlling when we cancel use of a read stream to do + * prefetching + */ +#define INDEX_SCAN_MIN_DISTANCE_NBATCHES 20 +#define INDEX_SCAN_MIN_TUPLE_DISTANCE 7 + +#define INDEX_SCAN_BATCH_COUNT(scan) \ + ((scan)->batchState->nextBatch - (scan)->batchState->headBatch) + +/* Did we already load batch with the requested index? */ +/* XXX shouldn't this also compare headBatch? maybe the batch was freed? */ +#define INDEX_SCAN_BATCH_LOADED(scan, idx) \ + ((idx) < (scan)->batchState->nextBatch) + +/* Have we loaded the maximum number of batches? */ +#define INDEX_SCAN_BATCH_FULL(scan) \ + (INDEX_SCAN_BATCH_COUNT(scan) == scan->batchState->maxBatches) + +/* Return batch for the provided index. */ +/* XXX Should this have an assert to enforce the batch is loaded? Maybe the + * index is too far back, but there happens to be a batch in the right slot? + * Could easily happen if we have to keep many batches around. + */ +#define INDEX_SCAN_BATCH(scan, idx) \ + ((scan)->batchState->batches[(idx) % INDEX_SCAN_MAX_BATCHES]) + +/* Is the position invalid/undefined? */ +#define INDEX_SCAN_POS_INVALID(pos) \ + (((pos)->batch == -1) && ((pos)->index == -1)) + +#ifdef INDEXAM_DEBUG +#define DEBUG_LOG(...) elog(AmRegularBackendProcess() ? NOTICE : DEBUG2, __VA_ARGS__) +#else +#define DEBUG_LOG(...) +#endif + +/* debug: print info about current batches */ +static void +index_batch_print(const char *label, IndexScanDesc scan) +{ +#ifdef INDEXAM_DEBUG + IndexScanBatchState *batches = scan->batchState; + + if (!scan->batchState) + return; + + if (!AmRegularBackendProcess()) + return; + if (IsCatalogRelation(scan->indexRelation)) + return; + + DEBUG_LOG("%s: batches headBatch %d nextBatch %d maxBatches %d", + label, + batches->headBatch, batches->nextBatch, batches->maxBatches); + + for (int i = batches->headBatch; i < batches->nextBatch; i++) + { + IndexScanBatchData *batch = INDEX_SCAN_BATCH(scan, i); + BTScanPos pos = (BTScanPos) batch->pos; + + DEBUG_LOG(" batch %d currPage %u %p firstItem %d lastItem %d killed %d", + i, pos->currPage, batch, batch->firstItem, batch->lastItem, + batch->numKilled); + } +#endif +} /* ---------------------------------------------------------------- * index_ interface functions @@ -283,6 +424,9 @@ index_beginscan(Relation heapRelation, scan->xs_snapshot = snapshot; scan->instrument = instrument; + if (indexRelation->rd_indam->amgetbatch != NULL) + index_batch_init(scan); + /* prepare to fetch index matches from table */ scan->xs_heapfetch = table_index_fetch_begin(heapRelation); @@ -380,6 +524,12 @@ index_rescan(IndexScanDesc scan, scan->kill_prior_tuple = false; /* for safety */ scan->xs_heap_continue = false; + /* + * Reset the batching. This makes it look like there are no batches, + * discards reads already scheduled within the read stream, etc. + */ + index_batch_reset(scan, true); + scan->indexRelation->rd_indam->amrescan(scan, keys, nkeys, orderbys, norderbys); } @@ -394,6 +544,9 @@ index_endscan(IndexScanDesc scan) SCAN_CHECKS; CHECK_SCAN_PROCEDURE(amendscan); + /* Cleanup batching, so that the AM can release pins and so on. */ + index_batch_end(scan); + /* Release resources (like buffer pins) from table accesses */ if (scan->xs_heapfetch) { @@ -421,10 +574,42 @@ index_endscan(IndexScanDesc scan) void index_markpos(IndexScanDesc scan) { + IndexScanBatchState *batchState = scan->batchState; + IndexScanBatchPos *markPos = &batchState->markPos; + IndexScanBatchData *markBatch = batchState->markBatch; + SCAN_CHECKS; - CHECK_SCAN_PROCEDURE(ammarkpos); - scan->indexRelation->rd_indam->ammarkpos(scan); + /* + * FIXME this should probably check there actually is a batch state. For + * now it works the only AM with mark/restore support is btree, and that + * has batching. But we should not rely on that, right? + */ + + /* + * Free the previous mark batch (if any), but only if the batch is no + * longer valid (in the current head/next range). This means that if we're + * marking the same batch (different item), we don't really do anything. + * + * XXX Should have some macro for this check, I guess. + */ + if (markBatch != NULL && (markPos->batch < batchState->headBatch || + markPos->batch >= batchState->nextBatch)) + { + batchState->markBatch = NULL; + index_batch_free(scan, markBatch); + } + + /* just copy the read position (which has to be valid) */ + batchState->markPos = batchState->readPos; + batchState->markBatch = INDEX_SCAN_BATCH(scan, batchState->markPos.batch); + + /* + * FIXME we need to make sure the batch does not get freed during the + * regular advances. + */ + + AssertCheckBatchPosValid(scan, &batchState->markPos); } /* ---------------- @@ -445,19 +630,60 @@ index_markpos(IndexScanDesc scan) void index_restrpos(IndexScanDesc scan) { + IndexScanBatchState *batchState; + IndexScanBatchPos *markPos; + IndexScanBatchData *markBatch; + Assert(IsMVCCSnapshot(scan->xs_snapshot)); SCAN_CHECKS; - CHECK_SCAN_PROCEDURE(amrestrpos); + CHECK_SCAN_PROCEDURE(amgetbatch); + CHECK_SCAN_PROCEDURE(amposreset); - /* release resources (like buffer pins) from table accesses */ + /* + * release resources (like buffer pins) from table accesses + * + * XXX: Currently, the distance is always remembered across any + * read_stream_reset calls (to work around the scan->batchState->reset + * behavior of resetting the stream to deal with running out of batches). + * We probably _should_ be forgetting the distance when we reset the + * stream here (through our table_index_fetch_reset call), though. + */ if (scan->xs_heapfetch) table_index_fetch_reset(scan->xs_heapfetch); scan->kill_prior_tuple = false; /* for safety */ scan->xs_heap_continue = false; - scan->indexRelation->rd_indam->amrestrpos(scan); + /* + * FIXME this should probably check there actually is a batch state. For + * now it works the only AM with mark/restore support is btree, and that + * has batching. But we should not rely on that, right? + */ + + batchState = scan->batchState; + markPos = &batchState->markPos; + markBatch = scan->batchState->markBatch; + + /* + * Call amposreset to let index AM know to invalidate any private state + * that independently tracks the scan's progress + */ + scan->indexRelation->rd_indam->amposreset(scan, markBatch); + + /* + * Reset the batching state, except for the marked batch, and make it look + * like we have a single batch -- the marked one. + */ + index_batch_reset(scan, false); + + batchState->markPos = *markPos; + batchState->readPos = *markPos; + batchState->headBatch = markPos->batch; + batchState->nextBatch = (batchState->headBatch + 1); + + INDEX_SCAN_BATCH(scan, batchState->markPos.batch) = markBatch; + batchState->markBatch = markBatch; /* also remember this */ } /* @@ -579,6 +805,17 @@ index_parallelrescan(IndexScanDesc scan) if (scan->xs_heapfetch) table_index_fetch_reset(scan->xs_heapfetch); + /* + * Reset the batching. This makes it look like there are no batches, + * discards reads already scheduled to the read stream, etc. + * + * XXX We do this before calling amparallelrescan, so that it could + * reinitialize everything (this probably does not matter very much, now + * that we've moved all the batching logic to indexam.c, it was more + * important when the index AM was responsible for more of it). + */ + index_batch_reset(scan, true); + /* amparallelrescan is optional; assume no-op if not provided by AM */ if (scan->indexRelation->rd_indam->amparallelrescan != NULL) scan->indexRelation->rd_indam->amparallelrescan(scan); @@ -614,6 +851,9 @@ index_beginscan_parallel(Relation heaprel, Relation indexrel, scan->xs_snapshot = snapshot; scan->instrument = instrument; + if (indexrel->rd_indam->amgetbatch != NULL) + index_batch_init(scan); + /* prepare to fetch index matches from table */ scan->xs_heapfetch = table_index_fetch_begin(heaprel); @@ -630,14 +870,286 @@ index_beginscan_parallel(Relation heaprel, Relation indexrel, ItemPointer index_getnext_tid(IndexScanDesc scan, ScanDirection direction) { - bool found; - SCAN_CHECKS; - CHECK_SCAN_PROCEDURE(amgettuple); /* XXX: we should assert that a snapshot is pushed or registered */ Assert(TransactionIdIsValid(RecentXmin)); + /* + * Index AMs that support plain index scans must provide exactly one of + * either the amgetbatch or amgettuple callbacks + */ + Assert(!(scan->indexRelation->rd_indam->amgettuple != NULL && + scan->indexRelation->rd_indam->amgetbatch != NULL)); + + if (scan->batchState != NULL) + return index_batch_getnext_tid(scan, direction); + else + return index_retail_getnext_tid(scan, direction); +} + +/* ---------------- + * index_getnext_batch_tid - ambatch index_getnext_tid implementation + * + * If we advance to the next batch, we release the previous one (unless it's + * tracked for mark/restore). + * + * If the scan direction changes, we release all batches except the current + * one (per readPos), to make it look it's the only batch we loaded. + * + * Returns the first/next TID, or NULL if no more items. + * + * FIXME This only sets xs_heaptid and xs_itup (if requested). Not sure if + * we need to do something with xs_hitup. Should this set xs_hitup? + * + * XXX Maybe if we advance the position to the next batch, we could keep the + * batch for a bit more, in case the scan direction changes (as long as it + * fits into maxBatches)? But maybe that's unnecessary complexity for too + * little gain, we'd need to be careful about releasing the batches lazily. + * ---------------- + */ +static ItemPointer +index_batch_getnext_tid(IndexScanDesc scan, ScanDirection direction) +{ + IndexScanBatchState *batchState = scan->batchState; + IndexScanBatchPos *readPos; + + CHECK_SCAN_PROCEDURE(amgetbatch); + + /* shouldn't get here without batching */ + AssertCheckBatches(scan); + + /* Initialize direction on first call */ + if (batchState->direction == NoMovementScanDirection) + batchState->direction = direction; + + /* + * Handle cancelling the use of the read stream for prefetching + */ + else if (unlikely(batchState->disabled && scan->xs_heapfetch->rs)) + { + index_batch_pos_reset(scan, &batchState->streamPos); + + read_stream_reset(scan->xs_heapfetch->rs); + scan->xs_heapfetch->rs = NULL; + } + + /* + * Handle change of scan direction (reset stream, ...). + * + * Release future batches properly, to make it look like the current batch + * is the only one we loaded. Also reset the stream position, as if we are + * just starting the scan. + */ + else if (unlikely(batchState->direction != direction)) + { + /* release "future" batches in the wrong direction */ + while (batchState->nextBatch > batchState->headBatch + 1) + { + IndexScanBatch fbatch; + + batchState->nextBatch--; + fbatch = INDEX_SCAN_BATCH(scan, batchState->nextBatch); + index_batch_free(scan, fbatch); + } + + /* + * Remember the new direction, and make sure the scan is not marked as + * "finished" (we might have already read the last batch, but now we + * need to start over). Do this before resetting the stream - it + * should not invoke the callback until the first read, but it may + * seem a bit confusing otherwise. + */ + batchState->direction = direction; + batchState->finished = false; + batchState->currentPrefetchBlock = InvalidBlockNumber; + + index_batch_pos_reset(scan, &batchState->streamPos); + if (scan->xs_heapfetch->rs) + read_stream_reset(scan->xs_heapfetch->rs); + } + + /* shortcut for the read position, for convenience */ + readPos = &batchState->readPos; + + DEBUG_LOG("index_batch_getnext_tid readPos %d %d direction %d", + readPos->batch, readPos->index, direction); + + /* + * Try advancing the batch position. If that doesn't succeed, it means we + * don't have more items in the current batch, and there's no future batch + * loaded. So try loading another batch, and maybe retry. + * + * FIXME This loop shouldn't happen more than twice. Maybe we should have + * some protection against infinite loops? To detect cases when the + * advance/getnext functions get to disagree? + */ + while (true) + { + /* + * If we manage to advance to the next items, return it and we're + * done. Otherwise try loading another batch. + */ + if (index_batch_pos_advance(scan, readPos, direction)) + { + IndexScanBatchData *readBatch = INDEX_SCAN_BATCH(scan, readPos->batch); + + /* set the TID / itup for the scan */ + scan->xs_heaptid = readBatch->items[readPos->index].heapTid; + if (scan->xs_want_itup) + scan->xs_itup = + (IndexTuple) (readBatch->currTuples + + readBatch->items[readPos->index].tupleOffset); + + DEBUG_LOG("readBatch %p firstItem %d lastItem %d readPos %d/%d TID (%u,%u)", + readBatch, readBatch->firstItem, readBatch->lastItem, + readPos->batch, readPos->index, + ItemPointerGetBlockNumber(&scan->xs_heaptid), + ItemPointerGetOffsetNumber(&scan->xs_heaptid)); + + /* + * If we advanced to the next batch, release the batch we no + * longer need. The positions is the "read" position, and we can + * compare it to headBatch. + */ + if (unlikely(readPos->batch != batchState->headBatch)) + { + IndexScanBatchData *headBatch = INDEX_SCAN_BATCH(scan, + batchState->headBatch); + + /* + * XXX When advancing readPos, the streamPos may get behind as + * we're only advancing it when actually requesting heap + * blocks. But we may not do that often enough - e.g. IOS may + * not need to access all-visible heap blocks, so the + * read_next callback does not get invoked for a long time. + * It's possible the stream gets so mucu behind the position + * gets invalid, as we already removed the batch. But that + * means we don't need any heap blocks until the current read + * position - if we did, we would not be in this situation (or + * it's a sign of a bug, as those two places are expected to + * be in sync). So if the streamPos still points at the batch + * we're about to free, just reset the position - we'll set it + * to readPos in the read_next callback later. + * + * XXX This can happen after the queue gets full, we "pause" + * the stream, and then reset it to continue. But I think that + * just increases the probability of hitting the issue, it's + * just more chance to to not advance the streamPos, which + * depends on when we try to fetch the first heap block after + * calling read_stream_reset(). + * + * FIXME Simplify/clarify/shorten this comment. Can it + * actually happen, if we never pull from the stream in IOS? + * We probably don't look ahead for the first call. + */ + if (unlikely(batchState->streamPos.batch == batchState->headBatch)) + { + DEBUG_LOG("index_batch_pos_reset called early (streamPos.batch == headBatch)"); + index_batch_pos_reset(scan, &batchState->streamPos); + } + + DEBUG_LOG("index_batch_getnext_tid free headBatch %p headBatch %d nextBatch %d", + headBatch, batchState->headBatch, batchState->nextBatch); + + /* Free the head batch (except when it's markBatch) */ + index_batch_free(scan, headBatch); + + /* + * In any case, remove the batch from the regular queue, even + * if we kept it for mark/restore. + */ + batchState->headBatch++; + + DEBUG_LOG("index_batch_getnext_tid batch freed headBatch %d nextBatch %d", + batchState->headBatch, batchState->nextBatch); + + index_batch_print("index_batch_getnext_tid / free old batch", scan); + + /* we can't skip any batches */ + Assert(batchState->headBatch == readPos->batch); + } + + pgstat_count_index_tuples(scan->indexRelation, 1); + return &scan->xs_heaptid; + } + + /* + * We failed to advance, i.e. we ran out of currently loaded batches. + * So if we filled the queue, this is a good time to reset the stream + * (before we try loading the next batch). + */ + if (unlikely(batchState->reset)) + { + DEBUG_LOG("resetting read stream readPos %d,%d", + readPos->batch, readPos->index); + + batchState->reset = false; + batchState->currentPrefetchBlock = InvalidBlockNumber; + + /* + * Need to reset the stream position, it might be too far behind. + * Ultimately we want to set it to readPos, but we can't do that + * yet - readPos still point sat the old batch, so just reset it + * and we'll init it to readPos later in the callback. + */ + index_batch_pos_reset(scan, &batchState->streamPos); + + if (scan->xs_heapfetch->rs) + read_stream_reset(scan->xs_heapfetch->rs); + } + + /* + * Failed to advance the read position, so try reading the next batch. + * If this fails, we're done - there's nothing more to load. + * + * Most of the batches should be loaded from read_stream_next_buffer, + * but we need to call index_batch_getnext here too, for two reasons. + * First, the read_stream only gets working after we try fetching the + * first heap tuple, so we need to load the initial batch (the head). + * Second, while most batches will be preloaded by the stream thanks + * to prefetching, it's possible to set effective_io_concurrency=0, + * and in that case all the batches get loaded from here. + */ + if (!index_batch_getnext(scan, direction)) + break; + + DEBUG_LOG("loaded next batch, retry to advance position"); + } + + /* + * If we get here, we failed to advance the position and there are no more + * batches, so we're done. + */ + DEBUG_LOG("no more batches to process"); + + /* + * Reset the position - we must not keep the last valid position, in case + * we change direction of the scan and start scanning again. If we kept + * the position, we'd skip the first item. + * + * XXX This is a bit strange. Do we really need to reset the position + * after returning the last item? I wonder if it means the API is not + * quite right. + */ + index_batch_pos_reset(scan, readPos); + + return NULL; +} + +/* ---------------- + * index_retail_getnext_tid - amgettuple index_getnext_tid implementation + * + * Returns the first/next TID, or NULL if no more items. + * ---------------- + */ +static ItemPointer +index_retail_getnext_tid(IndexScanDesc scan, ScanDirection direction) +{ + bool found; + + CHECK_SCAN_PROCEDURE(amgettuple); + /* * The AM's amgettuple proc finds the next index entry matching the scan * keys, and puts the TID into scan->xs_heaptid. It should also set @@ -704,9 +1216,18 @@ index_fetch_heap(IndexScanDesc scan, TupleTableSlot *slot) * amgettuple call, in index_getnext_tid). We do not do this when in * recovery because it may violate MVCC to do so. See comments in * RelationGetIndexScan(). + * + * XXX For scans using batching, record the flag in the batch (we will + * pass it to the AM later, when freeing it). Otherwise just pass it to + * the AM using the kill_prior_tuple field. */ if (!scan->xactStartedInRecovery) - scan->kill_prior_tuple = all_dead; + { + if (scan->batchState == NULL) + scan->kill_prior_tuple = all_dead; + else if (all_dead) + index_batch_kill_item(scan); + } return found; } @@ -1089,3 +1610,789 @@ index_opclass_options(Relation indrel, AttrNumber attnum, Datum attoptions, return build_local_reloptions(&relopts, attoptions, validate); } + +/* + * Check that a position (batch,item) is valid with respect to the batches we + * have currently loaded. + * + * XXX The "marked" batch is an exception. The marked batch may get outside + * the range of current batches, so make sure to never check the position + * for that. + */ +static void +AssertCheckBatchPosValid(IndexScanDesc scan, IndexScanBatchPos *pos) +{ +#ifdef USE_ASSERT_CHECKING + IndexScanBatchState *batchState = scan->batchState; + + /* make sure the position is valid for currently loaded batches */ + Assert(pos->batch >= batchState->headBatch); + Assert(pos->batch < batchState->nextBatch); +#endif +} + +/* + * Check a single batch is valid. + */ +static void +AssertCheckBatch(IndexScanDesc scan, IndexScanBatch batch) +{ +#ifdef USE_ASSERT_CHECKING + /* there must be valid range of items */ + Assert(batch->firstItem <= batch->lastItem); + Assert(batch->firstItem >= 0); + + /* we should have items (buffer and pointers) */ + Assert(batch->items != NULL); + + /* + * The number of killed items must be valid, and there must be an array of + * indexes if there are items. + */ + Assert(batch->numKilled >= 0); + Assert(!(batch->numKilled > 0 && batch->killedItems == NULL)); + + /* XXX can we check some of the other batch fields? */ +#endif +} + +/* + * Check invariants on current batches + * + * Makes sure the indexes are set as expected, the buffer size is within + * limits, and so on. + */ +static void +AssertCheckBatches(IndexScanDesc scan) +{ +#ifdef USE_ASSERT_CHECKING + IndexScanBatchState *batchState = scan->batchState; + + /* we should have batches initialized */ + Assert(batchState != NULL); + + /* We should not have too many batches. */ + Assert(batchState->maxBatches > 0 && + batchState->maxBatches <= INDEX_SCAN_MAX_BATCHES); + + /* + * The head/next indexes should define a valid range (in the cyclic + * buffer, and should not overflow maxBatches. + */ + Assert(batchState->headBatch >= 0 && + batchState->headBatch <= batchState->nextBatch); + Assert(batchState->nextBatch - batchState->headBatch <= + batchState->maxBatches); + + /* Check all current batches */ + for (int i = batchState->headBatch; i < batchState->nextBatch; i++) + { + IndexScanBatch batch = INDEX_SCAN_BATCH(scan, i); + + AssertCheckBatch(scan, batch); + } +#endif +} + +/* + * index_batch_pos_advance + * Advance the position to the next item, depending on scan direction. + * + * Advance the position to the next item, either in the same batch or the + * following one (if already available). + * + * We can advance only if we already have some batches loaded, and there's + * either enough items in the current batch, or some more items in the + * subsequent batches. + * + * If this is the first advance (right after loading the initial/head batch), + * position is still undefined. Otherwise we expect the position to be valid. + * + * Returns true if the position was advanced, false otherwise. + * + * The position is guaranteed to be valid only after a successful advance. + * If an advance fails (false returned), the position can be invalid. + * + * XXX This seems like a good place to enforce some "invariants", e.g. + * that the positions are always valid. We should never get here with + * invalid position (so probably should be initialized as part of loading the + * initial/head batch), and then invalidated if advance fails. Could be tricky + * for the stream position, though, because it can get "lag" for IOS etc. + */ +static pg_attribute_always_inline bool +index_batch_pos_advance(IndexScanDesc scan, IndexScanBatchPos *pos, + ScanDirection direction) +{ + IndexScanBatchData *batch; + + /* make sure we have batching initialized and consistent */ + AssertCheckBatches(scan); + + /* should know direction by now */ + Assert(direction == scan->batchState->direction); + Assert(direction != NoMovementScanDirection); + + /* We can't advance if there are no batches available. */ + if (INDEX_SCAN_BATCH_COUNT(scan) == 0) + return false; + + /* + * If the position has not been advanced yet, it has to be right after we + * loaded the initial batch (must be the head batch). In that case just + * initialize it to the batch's first item (or its last item, when + * scanning backwards). + * + * XXX Maybe we should just explicitly initialize the postition after + * loading the initial batch, without having to go through the advance. + */ + if (INDEX_SCAN_POS_INVALID(pos)) + { + /* + * We should have loaded the scan's initial batch, or maybe we have + * changed the direction of the scan after scanning all the way to the + * end (in which case the position is invalid, and we make it look + * like there is just one batch). We should have just one batch, + * though. + * + * XXX Actually, could there be more batches? Maybe we prefetched more + * batches right away? It doesn't seem to be a substantial invariant. + */ + Assert(INDEX_SCAN_BATCH_COUNT(scan) == 1); + + /* + * Get the initial batch (which must be the head), and initialize the + * position to the appropriate item for the current scan direction + */ + batch = INDEX_SCAN_BATCH(scan, scan->batchState->headBatch); + + pos->batch = scan->batchState->headBatch; + + if (ScanDirectionIsForward(direction)) + pos->index = batch->firstItem; + else + pos->index = batch->lastItem; + + AssertCheckBatchPosValid(scan, pos); + + return true; + } + + /* + * The position is already defined, so we should have some batches loaded + * and the position has to be valid with respect to those. + */ + AssertCheckBatchPosValid(scan, pos); + + /* + * Advance to the next item in the same batch, if there are more items. If + * we're at the last item, we'll try advancing to the next batch later. + */ + batch = INDEX_SCAN_BATCH(scan, pos->batch); + + if (ScanDirectionIsForward(direction)) + { + if (++pos->index <= batch->lastItem) + { + AssertCheckBatchPosValid(scan, pos); + + return true; + } + } + else /* ScanDirectionIsBackward */ + { + if (--pos->index >= batch->firstItem) + { + AssertCheckBatchPosValid(scan, pos); + + return true; + } + } + + /* + * We couldn't advance within the same batch, try advancing to the next + * batch, if it's already loaded. + */ + if (INDEX_SCAN_BATCH_LOADED(scan, pos->batch + 1)) + { + /* advance to the next batch */ + pos->batch++; + + batch = INDEX_SCAN_BATCH(scan, pos->batch); + Assert(batch != NULL); + + if (ScanDirectionIsForward(direction)) + pos->index = batch->firstItem; + else + pos->index = batch->lastItem; + + AssertCheckBatchPosValid(scan, pos); + + return true; + } + + /* can't advance */ + return false; +} + +/* + * index_batch_pos_reset + * Reset the position, so that it looks as if never advanced. + */ +static void +index_batch_pos_reset(IndexScanDesc scan, IndexScanBatchPos *pos) +{ + pos->batch = -1; + pos->index = -1; +} + +/* + * index_scan_stream_read_next + * return the next block to pass to the read stream + * + * This assumes the "current" scan direction, requested by the caller. + * + * If the direction changes before consuming all blocks, we'll reset the stream + * and start from scratch. The scan direction change is handled elsewhere. Here + * we rely on having the correct value in batchState->direction. + * + * The position of the read_stream is stored in streamPos, which may be ahead of + * the current readPos (which is what got consumed by the scan). + * + * The streamPos can however also get behind readPos too, when some blocks are + * skipped and not returned to the read_stream. An example is an index scan on + * a correlated index, with many duplicate blocks are skipped, or an IOS where + * all-visible blocks are skipped. + * + * The initial batch is always loaded from index_batch_getnext_tid(). We don't + * get here until the first read_stream_next_buffer() call, when pulling the + * first heap tuple from the stream. After that, most batches should be loaded + * by this callback, driven by the read_stream look-ahead distance. However, + * with disabled prefetching (that is, with effective_io_concurrency=0), all + * batches will be loaded in index_batch_getnext_tid. + * + * It's possible we got here only fairly late in the scan, e.g. if many tuples + * got skipped in the index-only scan, etc. In this case just use the read + * position as a streamPos starting point. + * + * XXX It seems the readPos/streamPos comments should be placed elsewhere. The + * read_stream callback does not seem like the right place. + */ +static BlockNumber +index_scan_stream_read_next(ReadStream *stream, + void *callback_private_data, + void *per_buffer_data) +{ + IndexScanDesc scan = (IndexScanDesc) callback_private_data; + IndexScanBatchState *batchState = scan->batchState; + IndexScanBatchPos *streamPos = &batchState->streamPos; + ScanDirection direction = batchState->direction; + + /* By now we should know the direction of the scan. */ + Assert(direction != NoMovementScanDirection); + + /* + * The read position (readPos) has to be valid. + * + * We initialize/advance it before even attempting to read the heap tuple, + * and it gets invalidated when we reach the end of the scan (but then we + * don't invoke the callback again). + * + * XXX This applies to the readPos. We'll use streamPos to determine which + * blocks to pass to the stream, and readPos may be used to initialize it. + */ + AssertCheckBatchPosValid(scan, &batchState->readPos); + + /* + * Try to advance the streamPos to the next item, and if that doesn't + * succeed (if there are no more items in loaded batches), try loading the + * next one. + * + * FIXME Unlike index_batch_getnext_tid, this can loop more than twice. If + * many blocks get skipped due to currentPrefetchBlock or all-visibility + * (per the "prefetch" callback), we get to load additional batches. In + * the worst case we hit the INDEX_SCAN_MAX_BATCHES limit and have to + * "pause" the stream. + */ + while (true) + { + bool advanced = false; + + /* + * If the stream position has not been initialized yet, set it to the + * current read position. This is the item the caller is tring to + * read, so it's what we should return to the stream. + */ + if (INDEX_SCAN_POS_INVALID(streamPos)) + { + *streamPos = batchState->readPos; + advanced = true; + } + else if (index_batch_pos_advance(scan, streamPos, direction)) + { + advanced = true; + } + + /* + * FIXME Maybe check the streamPos is not behind readPos? + * + * FIXME Actually, could streamPos get stale/lagging behind readPos, + * and if yes how much. Could it get so far behind to not be valid, + * pointing at a freed batch? In that case we can't even advance it, + * and we should just initialize it to readPos. We might do that + * anyway, I guess, just to save on "pointless" advances (it must + * agree with readPos, we can't allow "retroactively" changing the + * block sequence). + */ + + /* + * If we advanced the position, either return the block for the TID, + * or skip it (and then try advancing again). + * + * The block may be "skipped" for two reasons. First, the caller may + * define a "prefetch" callback that tells us to skip items (IOS does + * this to skip all-visible pages). Second, currentPrefetchBlock is + * used to skip duplicate block numbers (a sequence of TIDS for the + * same block). + */ + if (advanced) + { + IndexScanBatch streamBatch = INDEX_SCAN_BATCH(scan, streamPos->batch); + ItemPointer tid = &streamBatch->items[streamPos->index].heapTid; + + DEBUG_LOG("index_scan_stream_read_next: index %d TID (%u,%u)", + streamPos->index, + ItemPointerGetBlockNumber(tid), + ItemPointerGetOffsetNumber(tid)); + + /* + * If there's a prefetch callback, use it to decide if we need to + * read the next block. + * + * We need to do this before checking currentPrefetchBlock; it's + * essential that the VM cache used by index-only scans is + * intialized here. + */ + if (batchState->prefetch && + !batchState->prefetch(scan, batchState->prefetchArg, streamPos)) + { + DEBUG_LOG("index_scan_stream_read_next: skip block (callback)"); + continue; + } + + /* same block as before, don't need to read it */ + if (batchState->currentPrefetchBlock == ItemPointerGetBlockNumber(tid)) + { + DEBUG_LOG("index_scan_stream_read_next: skip block (currentPrefetchBlock)"); + continue; + } + + batchState->currentPrefetchBlock = ItemPointerGetBlockNumber(tid); + + return batchState->currentPrefetchBlock; + } + + /* + * Couldn't advance the position, no more items in the loaded batches. + * Try loading the next batch - if that succeeds, try advancing again + * (this time the advance should work, but we may skip all the items). + * + * If we fail to load the next batch, we're done. + */ + if (!index_batch_getnext(scan, direction)) + break; + + /* + * Consider disabling prefetching when we can't keep a sufficiently + * large "index tuple distance" between readPos and streamPos. + * + * Only consider doing this when we're not on the scan's initial + * batch, when readPos and streamPos share the same batch. + */ + if (!batchState->finished && !batchState->prefetchingLockedIn) + { + int indexdiff; + + if (streamPos->batch <= INDEX_SCAN_MIN_DISTANCE_NBATCHES) + { + /* Too early to check if prefetching should be disabled */ + } + else if (batchState->readPos.batch == streamPos->batch) + { + IndexScanBatchPos *readPos = &batchState->readPos; + + if (ScanDirectionIsForward(direction)) + indexdiff = streamPos->index - readPos->index; + else + { + IndexScanBatch readBatch = + INDEX_SCAN_BATCH(scan, readPos->batch); + + indexdiff = (readPos->index - readBatch->firstItem) - + (streamPos->index - readBatch->firstItem); + } + + if (indexdiff < INDEX_SCAN_MIN_TUPLE_DISTANCE) + { + batchState->disabled = true; + return InvalidBlockNumber; + } + else + { + batchState->prefetchingLockedIn = true; + } + } + else + batchState->prefetchingLockedIn = true; + } + } + + /* no more items in this scan */ + return InvalidBlockNumber; +} + +/* ---------------- + * index_batch_getnext - get the next batch of TIDs from a scan + * + * Returns true if we managed to read a batch of TIDs, or false if there are no + * more TIDs in the scan. The load may also return false if we used the maximum + * number of batches (INDEX_SCAN_MAX_BATCHES), in which case we'll reset the + * stream and continue the scan later. + * + * Returns true if the batch was loaded successfully, false otherwise. + * + * This only loads the TIDs and resets the various batch fields to fresh + * state. It does not set xs_heaptid/xs_itup/xs_hitup, that's the + * responsibility of the following index_batch_getnext_tid() calls. + * ---------------- + */ +static bool +index_batch_getnext(IndexScanDesc scan, ScanDirection direction) +{ + IndexScanBatchState *batchState = scan->batchState; + IndexScanBatch priorbatch = NULL, + batch = NULL; + + SCAN_CHECKS; + CHECK_SCAN_PROCEDURE(amgetbatch); + + /* XXX: we should assert that a snapshot is pushed or registered */ + Assert(TransactionIdIsValid(RecentXmin)); + + /* Did we already read the last batch for this scan? */ + if (batchState->finished) + return false; + + /* + * If we already used the maximum number of batch slots available, it's + * pointless to try loading another one. This can happen for various + * reasons, e.g. for index-only scans on all-visible table, or skipping + * duplicate blocks on perfectly correlated indexes, etc. + * + * We could enlarge the array to allow more batches, but that's futile, we + * can always construct a case using more memory. Not only it would risk + * OOM, it'd also be inefficient because this happens early in the scan + * (so it'd interfere with LIMIT queries). + */ + if (INDEX_SCAN_BATCH_FULL(scan)) + { + DEBUG_LOG("index_batch_getnext: ran out of space for batches"); + scan->batchState->reset = true; + return false; + } + + index_batch_print("index_batch_getnext / start", scan); + + /* + * Check if there's an existing batch that amgetbatch has to pick things + * up from + */ + if (batchState->headBatch < batchState->nextBatch) + priorbatch = INDEX_SCAN_BATCH(scan, batchState->nextBatch - 1); + + batch = scan->indexRelation->rd_indam->amgetbatch(scan, priorbatch, + direction); + if (batch != NULL) + { + /* + * We got the batch from the AM, but we need to add it to the queue. + * Maybe that should be part of the "batch allocation" that happens in + * the AM? + */ + int batchIndex = batchState->nextBatch; + + INDEX_SCAN_BATCH(scan, batchIndex) = batch; + + batchState->nextBatch++; + + DEBUG_LOG("index_batch_getnext headBatch %d nextBatch %d batch %p", + batchState->headBatch, batchState->nextBatch, batch); + + /* Delay initializing stream until reading from scan's second batch */ + if (priorbatch && !scan->xs_heapfetch->rs && !batchState->disabled && + enable_indexscan_prefetch) + scan->xs_heapfetch->rs = + read_stream_begin_relation(READ_STREAM_DEFAULT, NULL, + scan->heapRelation, MAIN_FORKNUM, + index_scan_stream_read_next, scan, 0); + } + else + batchState->finished = true; + + AssertCheckBatches(scan); + + index_batch_print("index_batch_getnext / end", scan); + + return (batch != NULL); +} + +/* + * index_batch_init + * Initialize various fields / arrays needed by batching. + * + * FIXME This is a bit ad-hoc hodge podge, due to how I was adding more and + * more pieces. Some of the fields may be not quite necessary, needs cleanup. + */ +static void +index_batch_init(IndexScanDesc scan) +{ + /* init batching info */ + Assert(scan->indexRelation->rd_indam->amgetbatch != NULL); + Assert(scan->indexRelation->rd_indam->amfreebatch != NULL); + + scan->batchState = palloc(sizeof(IndexScanBatchState)); + + /* + * Initialize the batch. + * + * We prefer to eagerly drop leaf page pins before amgetbatch returns. + * This avoids making VACUUM wait to acquire a cleanup lock on the page. + * + * We cannot safely drop leaf page pins during index-only scans due to a + * race condition involving VACUUM setting pages all-visible in the VM. + * It's also unsafe for plain index scans that use a non-MVCC snapshot. + * + * When we drop pins eagerly, the mechanism that marks index tuples as + * LP_DEAD has to deal with concurrent TID recycling races. The scheme + * used to detect unsafe TID recycling won't work when scanning unlogged + * relations (since it involves saving an affected page's LSN). Opt out + * of eager pin dropping during unlogged relation scans for now. + */ + scan->batchState->dropPin = + (!scan->xs_want_itup && IsMVCCSnapshot(scan->xs_snapshot) && + RelationNeedsWAL(scan->indexRelation)); + scan->batchState->finished = false; + scan->batchState->reset = false; + scan->batchState->prefetchingLockedIn = false; + scan->batchState->disabled = false; + scan->batchState->currentPrefetchBlock = InvalidBlockNumber; + scan->batchState->direction = NoMovementScanDirection; + /* positions in the queue of batches */ + index_batch_pos_reset(scan, &scan->batchState->readPos); + index_batch_pos_reset(scan, &scan->batchState->streamPos); + index_batch_pos_reset(scan, &scan->batchState->markPos); + + scan->batchState->markBatch = NULL; + scan->batchState->maxBatches = INDEX_SCAN_MAX_BATCHES; + scan->batchState->headBatch = 0; /* initial head batch */ + scan->batchState->nextBatch = 0; /* initial batch starts empty */ + + scan->batchState->batches = + palloc(sizeof(IndexScanBatchData *) * scan->batchState->maxBatches); + + scan->batchState->prefetch = NULL; + scan->batchState->prefetchArg = NULL; +} + +/* + * index_batch_reset + * Reset the batch before reading the next chunk of data. + * + * complete - true means we reset even marked batch + * + * XXX Should this reset the batch memory context, xs_itup, xs_hitup, etc? + */ +static void +index_batch_reset(IndexScanDesc scan, bool complete) +{ + IndexScanBatchState *batchState = scan->batchState; + + /* bail out if batching not enabled */ + if (!batchState) + return; + + AssertCheckBatches(scan); + + index_batch_print("index_batch_reset", scan); + + /* With batching enabled, we should have a read stream. Reset it. */ + Assert(scan->xs_heapfetch); + if (scan->xs_heapfetch->rs) + read_stream_reset(scan->xs_heapfetch->rs); + + /* reset the positions */ + index_batch_pos_reset(scan, &batchState->readPos); + index_batch_pos_reset(scan, &batchState->streamPos); + + /* + * With "complete" reset, make sure to also free the marked batch, either + * by just forgetting it (if it's still in the queue), or by explicitly + * freeing it. + * + * XXX Do this before the loop, so that it calls the amfreebatch(). + */ + if (complete && unlikely(batchState->markBatch != NULL)) + { + IndexScanBatchPos *markPos = &batchState->markPos; + IndexScanBatch markBatch = batchState->markBatch; + + /* always reset the position, forget the marked batch */ + batchState->markBatch = NULL; + + /* + * If we've already moved past the marked batch (it's not in the + * current queue), free it explicitly. Otherwise it'll be in the freed + * later. + */ + if (markPos->batch < batchState->headBatch || + markPos->batch >= batchState->nextBatch) + index_batch_free(scan, markBatch); + + /* reset position only after the queue range check */ + index_batch_pos_reset(scan, &batchState->markPos); + } + + /* release all currently loaded batches */ + while (batchState->headBatch < batchState->nextBatch) + { + IndexScanBatch batch = INDEX_SCAN_BATCH(scan, batchState->headBatch); + + DEBUG_LOG("freeing batch %d %p", batchState->headBatch, batch); + + index_batch_free(scan, batch); + + /* update the valid range, so that asserts / debugging works */ + batchState->headBatch++; + } + + /* reset relevant batch state fields */ + Assert(batchState->maxBatches == INDEX_SCAN_MAX_BATCHES); + batchState->headBatch = 0; /* initial batch */ + batchState->nextBatch = 0; /* initial batch is empty */ + + batchState->finished = false; + batchState->reset = false; + batchState->currentPrefetchBlock = InvalidBlockNumber; + + AssertCheckBatches(scan); +} + +static void +index_batch_kill_item(IndexScanDesc scan) +{ + IndexScanBatchPos *readPos = &scan->batchState->readPos; + IndexScanBatchData *readBatch = INDEX_SCAN_BATCH(scan, readPos->batch); + + AssertCheckBatchPosValid(scan, readPos); + + /* + * XXX Maybe we can move the state that indicates if an item has been + * killed into IndexScanBatchData.items[] array. + * + * See: + * https://postgr.es/m/CAH2-WznLN7P0i2-YEnv3QGmeA5AMjdcjkraO_nz3H2Va1V1WOA@mail.gmail.com + */ + if (readBatch->killedItems == NULL) + readBatch->killedItems = (int *) + palloc(MaxTIDsPerBTreePage * sizeof(int)); + if (readBatch->numKilled < MaxTIDsPerBTreePage) + readBatch->killedItems[readBatch->numKilled++] = readPos->index; +} + +static void +index_batch_free(IndexScanDesc scan, IndexScanBatch batch) +{ + SCAN_CHECKS; + CHECK_SCAN_PROCEDURE(amfreebatch); + + AssertCheckBatch(scan, batch); + + /* don't free the batch that is marked */ + if (batch == scan->batchState->markBatch) + return; + + scan->indexRelation->rd_indam->amfreebatch(scan, batch); +} + +/* */ +static void +index_batch_end(IndexScanDesc scan) +{ + index_batch_reset(scan, true); +} + +/* + * XXX Both index_batch_alloc() calls in btree use MaxTIDsPerBTreePage, + * which seems unfortunate - it increases the allocation sizes, even if + * the index would be fine with smaller arrays. This means all batches exceed + * ALLOC_CHUNK_LIMIT, forcing a separate malloc (expensive). + */ +IndexScanBatch +index_batch_alloc(int maxitems, bool want_itup) +{ + IndexScanBatch batch = palloc(offsetof(IndexScanBatchData, items) + + sizeof(IndexScanBatchPosItem) * maxitems); + + batch->firstItem = -1; + batch->lastItem = -1; + batch->killedItems = NULL; + batch->numKilled = 0; + + /* + * If we are doing an index-only scan, we need a tuple storage workspace. + * We allocate BLCKSZ for this, which should always give the index AM + * enough space to fit a full page's worth of tuples. + */ + batch->currTuples = NULL; + if (want_itup) + batch->currTuples = palloc(BLCKSZ); + + batch->buf = InvalidBuffer; + batch->pos = NULL; + batch->itemsvisibility = NULL; /* per-batch IOS visibility */ + + return batch; +} + +/* + * Unlock batch->buf. If batch scan is dropPin, drop the pin, too. Dropping + * the pin prevents VACUUM from blocking on acquiring a cleanup lock. + */ +void +index_batch_unlock(Relation rel, bool dropPin, IndexScanBatch batch) +{ + if (!dropPin) + { + if (!RelationUsesLocalBuffers(rel)) + VALGRIND_MAKE_MEM_NOACCESS(BufferGetPage(batch->buf), BLCKSZ); + + /* Just drop the lock (not the pin) */ + LockBuffer(batch->buf, BUFFER_LOCK_UNLOCK); + return; + } + + /* + * Drop both the lock and the pin. + * + * Have to set batch->lsn so that amfreebatch has a way to detect when + * concurrent heap TID recycling by VACUUM might have taken place. It'll + * only be safe to set any index tuple LP_DEAD bits when the page LSN + * hasn't advanced. + */ + Assert(RelationNeedsWAL(rel)); + batch->lsn = BufferGetLSNAtomic(batch->buf); + LockBuffer(batch->buf, BUFFER_LOCK_UNLOCK); + if (!RelationUsesLocalBuffers(rel)) + VALGRIND_MAKE_MEM_NOACCESS(BufferGetPage(batch->buf), BLCKSZ); + ReleaseBuffer(batch->buf); + batch->buf = InvalidBuffer; /* defensive */ +} diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c index fdff960c1302..fba562df8a09 100644 --- a/src/backend/access/nbtree/nbtree.c +++ b/src/backend/access/nbtree/nbtree.c @@ -158,11 +158,12 @@ bthandler(PG_FUNCTION_ARGS) amroutine->amadjustmembers = btadjustmembers; amroutine->ambeginscan = btbeginscan; amroutine->amrescan = btrescan; - amroutine->amgettuple = btgettuple; + amroutine->amgettuple = NULL; + amroutine->amgetbatch = btgetbatch; + amroutine->amfreebatch = btfreebatch; amroutine->amgetbitmap = btgetbitmap; amroutine->amendscan = btendscan; - amroutine->ammarkpos = btmarkpos; - amroutine->amrestrpos = btrestrpos; + amroutine->amposreset = btposreset; amroutine->amestimateparallelscan = btestimateparallelscan; amroutine->aminitparallelscan = btinitparallelscan; amroutine->amparallelrescan = btparallelrescan; @@ -220,13 +221,12 @@ btinsert(Relation rel, Datum *values, bool *isnull, } /* - * btgettuple() -- Get the next tuple in the scan. + * btgetbatch() -- Get the next batch of tuples in the scan. */ -bool -btgettuple(IndexScanDesc scan, ScanDirection dir) +IndexScanBatch +btgetbatch(IndexScanDesc scan, IndexScanBatch batch, ScanDirection dir) { BTScanOpaque so = (BTScanOpaque) scan->opaque; - bool res; Assert(scan->heapRelation != NULL); @@ -241,44 +241,18 @@ btgettuple(IndexScanDesc scan, ScanDirection dir) * the appropriate direction. If we haven't done so yet, we call * _bt_first() to get the first item in the scan. */ - if (!BTScanPosIsValid(so->currPos)) - res = _bt_first(scan, dir); + if (batch == NULL) + batch = _bt_first(scan, dir); else - { - /* - * Check to see if we should kill the previously-fetched tuple. - */ - if (scan->kill_prior_tuple) - { - /* - * Yes, remember it for later. (We'll deal with all such - * tuples at once right before leaving the index page.) The - * test for numKilled overrun is not just paranoia: if the - * caller reverses direction in the indexscan then the same - * item might get entered multiple times. It's not worth - * trying to optimize that, so we don't detect it, but instead - * just forget any excess entries. - */ - if (so->killedItems == NULL) - so->killedItems = (int *) - palloc(MaxTIDsPerBTreePage * sizeof(int)); - if (so->numKilled < MaxTIDsPerBTreePage) - so->killedItems[so->numKilled++] = so->currPos.itemIndex; - } - - /* - * Now continue the scan. - */ - res = _bt_next(scan, dir); - } + batch = _bt_next(scan, dir, batch); - /* If we have a tuple, return it ... */ - if (res) + /* If we have a batch, return it ... */ + if (batch) break; /* ... otherwise see if we need another primitive index scan */ } while (so->numArrayKeys && _bt_start_prim_scan(scan, dir)); - return res; + return batch; } /* @@ -288,6 +262,7 @@ int64 btgetbitmap(IndexScanDesc scan, TIDBitmap *tbm) { BTScanOpaque so = (BTScanOpaque) scan->opaque; + IndexScanBatch batch; int64 ntids = 0; ItemPointer heapTid; @@ -296,29 +271,33 @@ btgetbitmap(IndexScanDesc scan, TIDBitmap *tbm) /* Each loop iteration performs another primitive index scan */ do { - /* Fetch the first page & tuple */ - if (_bt_first(scan, ForwardScanDirection)) + /* Fetch the first batch */ + if ((batch = _bt_first(scan, ForwardScanDirection))) { - /* Save tuple ID, and continue scanning */ - heapTid = &scan->xs_heaptid; + int itemIndex = 0; + + /* Save first tuple's TID */ + heapTid = &batch->items[itemIndex].heapTid; tbm_add_tuples(tbm, heapTid, 1, false); ntids++; for (;;) { - /* - * Advance to next tuple within page. This is the same as the - * easy case in _bt_next(). - */ - if (++so->currPos.itemIndex > so->currPos.lastItem) + /* Advance to next TID within page-sized batch */ + if (++itemIndex > batch->lastItem) { + /* btfreebatch won't be called */ + ReleaseBuffer(batch->buf); + /* let _bt_next do the heavy lifting */ - if (!_bt_next(scan, ForwardScanDirection)) + itemIndex = 0; + batch = _bt_next(scan, ForwardScanDirection, batch); + if (!batch) break; } /* Save tuple ID, and continue scanning */ - heapTid = &so->currPos.items[so->currPos.itemIndex].heapTid; + heapTid = &batch->items[itemIndex].heapTid; tbm_add_tuples(tbm, heapTid, 1, false); ntids++; } @@ -346,8 +325,6 @@ btbeginscan(Relation rel, int nkeys, int norderbys) /* allocate private workspace */ so = (BTScanOpaque) palloc(sizeof(BTScanOpaqueData)); - BTScanPosInvalidate(so->currPos); - BTScanPosInvalidate(so->markPos); if (scan->numberOfKeys > 0) so->keyData = (ScanKey) palloc(scan->numberOfKeys * sizeof(ScanKeyData)); else @@ -361,16 +338,6 @@ btbeginscan(Relation rel, int nkeys, int norderbys) so->orderProcs = NULL; so->arrayContext = NULL; - so->killedItems = NULL; /* until needed */ - so->numKilled = 0; - - /* - * We don't know yet whether the scan will be index-only, so we do not - * allocate the tuple workspace arrays until btrescan. However, we set up - * scan->xs_itupdesc whether we'll need it or not, since that's so cheap. - */ - so->currTuples = so->markTuples = NULL; - scan->xs_itupdesc = RelationGetDescr(rel); scan->opaque = so; @@ -387,80 +354,49 @@ btrescan(IndexScanDesc scan, ScanKey scankey, int nscankeys, { BTScanOpaque so = (BTScanOpaque) scan->opaque; - /* we aren't holding any read locks, but gotta drop the pins */ - if (BTScanPosIsValid(so->currPos)) - { - /* Before leaving current page, deal with any killed items */ - if (so->numKilled > 0) - _bt_killitems(scan); - BTScanPosUnpinIfPinned(so->currPos); - BTScanPosInvalidate(so->currPos); - } - /* - * We prefer to eagerly drop leaf page pins before btgettuple returns. - * This avoids making VACUUM wait to acquire a cleanup lock on the page. - * - * We cannot safely drop leaf page pins during index-only scans due to a - * race condition involving VACUUM setting pages all-visible in the VM. - * It's also unsafe for plain index scans that use a non-MVCC snapshot. - * - * When we drop pins eagerly, the mechanism that marks so->killedItems[] - * index tuples LP_DEAD has to deal with concurrent TID recycling races. - * The scheme used to detect unsafe TID recycling won't work when scanning - * unlogged relations (since it involves saving an affected page's LSN). - * Opt out of eager pin dropping during unlogged relation scans for now - * (this is preferable to opting out of kill_prior_tuple LP_DEAD setting). - * - * Also opt out of dropping leaf page pins eagerly during bitmap scans. - * Pins cannot be held for more than an instant during bitmap scans either - * way, so we might as well avoid wasting cycles on acquiring page LSNs. - * - * See nbtree/README section on making concurrent TID recycling safe. - * - * Note: so->dropPin should never change across rescans. + * Reset the scan keys */ - so->dropPin = (!scan->xs_want_itup && - IsMVCCSnapshot(scan->xs_snapshot) && - RelationNeedsWAL(scan->indexRelation) && - scan->heapRelation != NULL); - - so->markItemIndex = -1; + if (scankey && scan->numberOfKeys > 0) + memcpy(scan->keyData, scankey, scan->numberOfKeys * sizeof(ScanKeyData)); so->needPrimScan = false; so->scanBehind = false; so->oppositeDirCheck = false; - BTScanPosUnpinIfPinned(so->markPos); - BTScanPosInvalidate(so->markPos); + so->numberOfKeys = 0; /* until _bt_preprocess_keys sets it */ + so->numArrayKeys = 0; /* ditto */ +} +/* + * btfreebatch() -- Free batch, releasing its buffer pin + * + * XXX Should we really be freeing memory like this? What if we were to just + * reuse most memory across distinct pages, avoiding pfree/palloc cycles? + */ +void +btfreebatch(IndexScanDesc scan, IndexScanBatch batch) +{ /* - * Allocate tuple workspace arrays, if needed for an index-only scan and - * not already done in a previous rescan call. To save on palloc - * overhead, both workspaces are allocated as one palloc block; only this - * function and btendscan know that. - * - * NOTE: this data structure also makes it safe to return data from a - * "name" column, even though btree name_ops uses an underlying storage - * datatype of cstring. The risk there is that "name" is supposed to be - * padded to NAMEDATALEN, but the actual index tuple is probably shorter. - * However, since we only return data out of tuples sitting in the - * currTuples array, a fetch of NAMEDATALEN bytes can at worst pull some - * data out of the markTuples array --- running off the end of memory for - * a SIGSEGV is not possible. Yeah, this is ugly as sin, but it beats - * adding special-case treatment for name_ops elsewhere. + * Check if there are tuples to kill from this batch (that weren't already + * killed earlier on) */ - if (scan->xs_want_itup && so->currTuples == NULL) + if (batch->numKilled > 0) + _bt_killitems(scan, batch); + + if (batch->itemsvisibility) + pfree(batch->itemsvisibility); + + if (batch->currTuples) + pfree(batch->currTuples); + + if (batch->pos) { - so->currTuples = (char *) palloc(BLCKSZ * 2); - so->markTuples = so->currTuples + BLCKSZ; + if (!scan->batchState || !scan->batchState->dropPin) + ReleaseBuffer(batch->buf); + + pfree(batch->pos); } - /* - * Reset the scan keys - */ - if (scankey && scan->numberOfKeys > 0) - memcpy(scan->keyData, scankey, scan->numberOfKeys * sizeof(ScanKeyData)); - so->numberOfKeys = 0; /* until _bt_preprocess_keys sets it */ - so->numArrayKeys = 0; /* ditto */ + pfree(batch); } /* @@ -471,116 +407,50 @@ btendscan(IndexScanDesc scan) { BTScanOpaque so = (BTScanOpaque) scan->opaque; - /* we aren't holding any read locks, but gotta drop the pins */ - if (BTScanPosIsValid(so->currPos)) - { - /* Before leaving current page, deal with any killed items */ - if (so->numKilled > 0) - _bt_killitems(scan); - BTScanPosUnpinIfPinned(so->currPos); - } - - so->markItemIndex = -1; - BTScanPosUnpinIfPinned(so->markPos); - - /* No need to invalidate positions, the RAM is about to be freed. */ - /* Release storage */ if (so->keyData != NULL) pfree(so->keyData); /* so->arrayKeys and so->orderProcs are in arrayContext */ if (so->arrayContext != NULL) MemoryContextDelete(so->arrayContext); - if (so->killedItems != NULL) - pfree(so->killedItems); - if (so->currTuples != NULL) - pfree(so->currTuples); - /* so->markTuples should not be pfree'd, see btrescan */ pfree(so); } /* - * btmarkpos() -- save current scan position + * btposreset() -- invalidate scan's array keys */ void -btmarkpos(IndexScanDesc scan) +btposreset(IndexScanDesc scan, IndexScanBatch markbatch) { BTScanOpaque so = (BTScanOpaque) scan->opaque; + BTScanPos pos; - /* There may be an old mark with a pin (but no lock). */ - BTScanPosUnpinIfPinned(so->markPos); + if (!so->numArrayKeys) + return; /* - * Just record the current itemIndex. If we later step to next page - * before releasing the marked position, _bt_steppage makes a full copy of - * the currPos struct in markPos. If (as often happens) the mark is moved - * before we leave the page, we don't have to do that work. + * Core system is about to restore a mark associated with a previously + * returned batch. Reset the scan's arrays to make all this safe. */ - if (BTScanPosIsValid(so->currPos)) - so->markItemIndex = so->currPos.itemIndex; - else - { - BTScanPosInvalidate(so->markPos); - so->markItemIndex = -1; - } -} - -/* - * btrestrpos() -- restore scan to last saved position - */ -void -btrestrpos(IndexScanDesc scan) -{ - BTScanOpaque so = (BTScanOpaque) scan->opaque; + pos = (BTScanPos) markbatch->pos; + _bt_start_array_keys(scan, pos->dir); - if (so->markItemIndex >= 0) - { - /* - * The scan has never moved to a new page since the last mark. Just - * restore the itemIndex. - * - * NB: In this case we can't count on anything in so->markPos to be - * accurate. - */ - so->currPos.itemIndex = so->markItemIndex; - } + /* + * Core system will invalidate all other batches. + * + * Deal with this by unsetting needPrimScan as well as moreRight (or as + * well as moreLeft, when scanning backwards). That way, the next time + * _bt_next is called it will step to the right (or to the left). At that + * point _bt_readpage will restore the scan's arrays to elements that + * correctly track the next page's position in the index's key space. + */ + if (ScanDirectionIsForward(pos->dir)) + pos->moreRight = true; else - { - /* - * The scan moved to a new page after last mark or restore, and we are - * now restoring to the marked page. We aren't holding any read - * locks, but if we're still holding the pin for the current position, - * we must drop it. - */ - if (BTScanPosIsValid(so->currPos)) - { - /* Before leaving current page, deal with any killed items */ - if (so->numKilled > 0) - _bt_killitems(scan); - BTScanPosUnpinIfPinned(so->currPos); - } - - if (BTScanPosIsValid(so->markPos)) - { - /* bump pin on mark buffer for assignment to current buffer */ - if (BTScanPosIsPinned(so->markPos)) - IncrBufferRefCount(so->markPos.buf); - memcpy(&so->currPos, &so->markPos, - offsetof(BTScanPosData, items[1]) + - so->markPos.lastItem * sizeof(BTScanPosItem)); - if (so->currTuples) - memcpy(so->currTuples, so->markTuples, - so->markPos.nextTupleOffset); - /* Reset the scan's array keys (see _bt_steppage for why) */ - if (so->numArrayKeys) - { - _bt_start_array_keys(scan, so->currPos.dir); - so->needPrimScan = false; - } - } - else - BTScanPosInvalidate(so->currPos); - } + pos->moreLeft = true; + so->needPrimScan = false; + so->scanBehind = false; + so->oppositeDirCheck = false; } /* @@ -827,15 +697,6 @@ _bt_parallel_seize(IndexScanDesc scan, BlockNumber *next_scan_page, *next_scan_page = InvalidBlockNumber; *last_curr_page = InvalidBlockNumber; - /* - * Reset so->currPos, and initialize moreLeft/moreRight such that the next - * call to _bt_readnextpage treats this backend similarly to a serial - * backend that steps from *last_curr_page to *next_scan_page (unless this - * backend's so->currPos is initialized by _bt_readfirstpage before then). - */ - BTScanPosInvalidate(so->currPos); - so->currPos.moreLeft = so->currPos.moreRight = true; - if (first) { /* @@ -985,8 +846,6 @@ _bt_parallel_done(IndexScanDesc scan) BTParallelScanDesc btscan; bool status_changed = false; - Assert(!BTScanPosIsValid(so->currPos)); - /* Do nothing, for non-parallel scans */ if (parallel_scan == NULL) return; diff --git a/src/backend/access/nbtree/nbtsearch.c b/src/backend/access/nbtree/nbtsearch.c index 0605356ec9f0..54a767a4fe41 100644 --- a/src/backend/access/nbtree/nbtsearch.c +++ b/src/backend/access/nbtree/nbtsearch.c @@ -25,62 +25,33 @@ #include "utils/rel.h" -static inline void _bt_drop_lock_and_maybe_pin(Relation rel, BTScanOpaque so); static Buffer _bt_moveright(Relation rel, Relation heaprel, BTScanInsert key, Buffer buf, bool forupdate, BTStack stack, int access); static OffsetNumber _bt_binsrch(Relation rel, BTScanInsert key, Buffer buf); static int _bt_binsrch_posting(BTScanInsert key, Page page, OffsetNumber offnum); -static bool _bt_readpage(IndexScanDesc scan, ScanDirection dir, - OffsetNumber offnum, bool firstpage); -static void _bt_saveitem(BTScanOpaque so, int itemIndex, - OffsetNumber offnum, IndexTuple itup); -static int _bt_setuppostingitems(BTScanOpaque so, int itemIndex, +static bool _bt_readpage(IndexScanDesc scan, IndexScanBatch newbatch, + ScanDirection dir, OffsetNumber offnum, + bool firstpage); +static void _bt_saveitem(IndexScanBatch newbatch, int itemIndex, + OffsetNumber offnum, IndexTuple itup, + int *tupleOffset); +static int _bt_setuppostingitems(IndexScanBatch newbatch, int itemIndex, OffsetNumber offnum, const ItemPointerData *heapTid, - IndexTuple itup); -static inline void _bt_savepostingitem(BTScanOpaque so, int itemIndex, + IndexTuple itup, int *tupleOffset); +static inline void _bt_savepostingitem(IndexScanBatch newbatch, int itemIndex, OffsetNumber offnum, - ItemPointer heapTid, int tupleOffset); -static inline void _bt_returnitem(IndexScanDesc scan, BTScanOpaque so); -static bool _bt_steppage(IndexScanDesc scan, ScanDirection dir); -static bool _bt_readfirstpage(IndexScanDesc scan, OffsetNumber offnum, - ScanDirection dir); -static bool _bt_readnextpage(IndexScanDesc scan, BlockNumber blkno, - BlockNumber lastcurrblkno, ScanDirection dir, - bool seized); + ItemPointer heapTid, int baseOffset); +static IndexScanBatch _bt_readfirstpage(IndexScanDesc scan, IndexScanBatch firstbatch, + OffsetNumber offnum, ScanDirection dir); +static IndexScanBatch _bt_readnextpage(IndexScanDesc scan, BlockNumber blkno, + BlockNumber lastcurrblkno, + ScanDirection dir, bool firstpage); static Buffer _bt_lock_and_validate_left(Relation rel, BlockNumber *blkno, BlockNumber lastcurrblkno); -static bool _bt_endpoint(IndexScanDesc scan, ScanDirection dir); - - -/* - * _bt_drop_lock_and_maybe_pin() - * - * Unlock so->currPos.buf. If scan is so->dropPin, drop the pin, too. - * Dropping the pin prevents VACUUM from blocking on acquiring a cleanup lock. - */ -static inline void -_bt_drop_lock_and_maybe_pin(Relation rel, BTScanOpaque so) -{ - if (!so->dropPin) - { - /* Just drop the lock (not the pin) */ - _bt_unlockbuf(rel, so->currPos.buf); - return; - } - - /* - * Drop both the lock and the pin. - * - * Have to set so->currPos.lsn so that _bt_killitems has a way to detect - * when concurrent heap TID recycling by VACUUM might have taken place. - */ - Assert(RelationNeedsWAL(rel)); - so->currPos.lsn = BufferGetLSNAtomic(so->currPos.buf); - _bt_relbuf(rel, so->currPos.buf); - so->currPos.buf = InvalidBuffer; -} +static IndexScanBatch _bt_endpoint(IndexScanDesc scan, ScanDirection dir, + IndexScanBatch firstbatch); /* * _bt_search() -- Search the tree for a particular scankey, @@ -870,20 +841,16 @@ _bt_compare(Relation rel, * conditions, and the tree ordering. We find the first item (or, * if backwards scan, the last item) in the tree that satisfies the * qualifications in the scan key. On success exit, data about the - * matching tuple(s) on the page has been loaded into so->currPos. We'll - * drop all locks and hold onto a pin on page's buffer, except during - * so->dropPin scans, when we drop both the lock and the pin. - * _bt_returnitem sets the next item to return to scan on success exit. + * matching tuple(s) on the page has been loaded into the returned batch. * - * If there are no matching items in the index, we return false, with no - * pins or locks held. so->currPos will remain invalid. + * If there are no matching items in the index, we just return NULL. * * Note that scan->keyData[], and the so->keyData[] scankey built from it, * are both search-type scankeys (see nbtree/README for more about this). * Within this routine, we build a temporary insertion-type scankey to use * in locating the scan start position. */ -bool +IndexScanBatch _bt_first(IndexScanDesc scan, ScanDirection dir) { Relation rel = scan->indexRelation; @@ -897,8 +864,7 @@ _bt_first(IndexScanDesc scan, ScanDirection dir) StrategyNumber strat_total = InvalidStrategy; BlockNumber blkno = InvalidBlockNumber, lastcurrblkno; - - Assert(!BTScanPosIsValid(so->currPos)); + IndexScanBatch firstbatch; /* * Examine the scan keys and eliminate any redundant keys; also mark the @@ -923,7 +889,7 @@ _bt_first(IndexScanDesc scan, ScanDirection dir) */ if (scan->parallel_scan != NULL && !_bt_parallel_seize(scan, &blkno, &lastcurrblkno, true)) - return false; + return false; /* definitely done (so->needPrimscan is unset) */ /* * Initialize the scan's arrays (if any) for the current scan direction @@ -940,14 +906,8 @@ _bt_first(IndexScanDesc scan, ScanDirection dir) * _bt_readnextpage releases the scan for us (not _bt_readfirstpage). */ Assert(scan->parallel_scan != NULL); - Assert(!so->needPrimScan); - Assert(blkno != P_NONE); - - if (!_bt_readnextpage(scan, blkno, lastcurrblkno, dir, true)) - return false; - _bt_returnitem(scan, so); - return true; + return _bt_readnextpage(scan, blkno, lastcurrblkno, dir, true); } /* @@ -1239,6 +1199,10 @@ _bt_first(IndexScanDesc scan, ScanDirection dir) } } + /* Allocate space for first batch */ + firstbatch = index_batch_alloc(MaxTIDsPerBTreePage, scan->xs_want_itup); + firstbatch->pos = palloc(sizeof(BTScanPosData)); + /* * If we found no usable boundary keys, we have to start from one end of * the tree. Walk down that edge to the first or last key, and scan from @@ -1247,7 +1211,7 @@ _bt_first(IndexScanDesc scan, ScanDirection dir) * Note: calls _bt_readfirstpage for us, which releases the parallel scan. */ if (keysz == 0) - return _bt_endpoint(scan, dir); + return _bt_endpoint(scan, dir, firstbatch); /* * We want to start the scan somewhere within the index. Set up an @@ -1515,12 +1479,12 @@ _bt_first(IndexScanDesc scan, ScanDirection dir) * position ourselves on the target leaf page. */ Assert(ScanDirectionIsBackward(dir) == inskey.backward); - stack = _bt_search(rel, NULL, &inskey, &so->currPos.buf, BT_READ); + stack = _bt_search(rel, NULL, &inskey, &firstbatch->buf, BT_READ); /* don't need to keep the stack around... */ _bt_freestack(stack); - if (!BufferIsValid(so->currPos.buf)) + if (!BufferIsValid(firstbatch->buf)) { Assert(!so->needPrimScan); @@ -1536,11 +1500,11 @@ _bt_first(IndexScanDesc scan, ScanDirection dir) if (IsolationIsSerializable()) { PredicateLockRelation(rel, scan->xs_snapshot); - stack = _bt_search(rel, NULL, &inskey, &so->currPos.buf, BT_READ); + stack = _bt_search(rel, NULL, &inskey, &firstbatch->buf, BT_READ); _bt_freestack(stack); } - if (!BufferIsValid(so->currPos.buf)) + if (!BufferIsValid(firstbatch->buf)) { _bt_parallel_done(scan); return false; @@ -1548,11 +1512,11 @@ _bt_first(IndexScanDesc scan, ScanDirection dir) } /* position to the precise item on the page */ - offnum = _bt_binsrch(rel, &inskey, so->currPos.buf); + offnum = _bt_binsrch(rel, &inskey, firstbatch->buf); /* * Now load data from the first page of the scan (usually the page - * currently in so->currPos.buf). + * currently in firstbatch.buf). * * If inskey.nextkey = false and inskey.backward = false, offnum is * positioned at the first non-pivot tuple >= inskey.scankeys. @@ -1570,69 +1534,79 @@ _bt_first(IndexScanDesc scan, ScanDirection dir) * for the page. For example, when inskey is both < the leaf page's high * key and > all of its non-pivot tuples, offnum will be "maxoff + 1". */ - if (!_bt_readfirstpage(scan, offnum, dir)) - return false; - - _bt_returnitem(scan, so); - return true; + return _bt_readfirstpage(scan, firstbatch, offnum, dir); } /* * _bt_next() -- Get the next item in a scan. * - * On entry, so->currPos describes the current page, which may be pinned - * but is not locked, and so->currPos.itemIndex identifies which item was - * previously returned. + * On entry, priorbatch describes the batch that was last returned by + * btgetbatch. We'll use the prior batch's positioning information to + * decide which page to read next. * - * On success exit, so->currPos is updated as needed, and _bt_returnitem - * sets the next item to return to the scan. so->currPos remains valid. + * On success exit, returns the next batch. There must be at least one + * matching tuple on any returned batch (else we'd just return NULL). * - * On failure exit (no more tuples), we invalidate so->currPos. It'll - * still be possible for the scan to return tuples by changing direction, - * though we'll need to call _bt_first anew in that other direction. + * On failure exit (no more tuples), we return NULL. It'll still be + * possible for the scan to return tuples by changing direction, though + * we'll need to call _bt_first anew in that other direction. */ -bool -_bt_next(IndexScanDesc scan, ScanDirection dir) +IndexScanBatch +_bt_next(IndexScanDesc scan, ScanDirection dir, IndexScanBatch priorbatch) { BTScanOpaque so = (BTScanOpaque) scan->opaque; + BlockNumber blkno, + lastcurrblkno; + BTScanPos priorpos = (BTScanPos) priorbatch->pos; - Assert(BTScanPosIsValid(so->currPos)); + Assert(BTScanPosIsValid(*priorpos)); - /* - * Advance to next tuple on current page; or if there's no more, try to - * step to the next page with data. - */ + /* Walk to the next page with data */ if (ScanDirectionIsForward(dir)) - { - if (++so->currPos.itemIndex > so->currPos.lastItem) - { - if (!_bt_steppage(scan, dir)) - return false; - } - } + blkno = priorpos->nextPage; else + blkno = priorpos->prevPage; + lastcurrblkno = priorpos->currPage; + + /* + * Cancel primitive index scans that were scheduled when the call to + * _bt_readpage for pos happened to use the opposite direction to the one + * that we're stepping in now. (It's okay to leave the scan's array keys + * as-is, since the next _bt_readpage will advance them.) + */ + if (priorpos->dir != dir) + so->needPrimScan = false; + + if (blkno == P_NONE || + (ScanDirectionIsForward(dir) ? + !priorpos->moreRight : !priorpos->moreLeft)) { - if (--so->currPos.itemIndex < so->currPos.firstItem) - { - if (!_bt_steppage(scan, dir)) - return false; - } + /* + * priorpos _bt_readpage call ended scan in this direction (though if + * so->needPrimScan was set the scan will continue in _bt_first) + */ + _bt_parallel_done(scan); + return NULL; } - _bt_returnitem(scan, so); - return true; + /* parallel scan must seize the scan to get next blkno */ + if (scan->parallel_scan != NULL && + !_bt_parallel_seize(scan, &blkno, &lastcurrblkno, false)) + return NULL; /* done iff so->needPrimScan wasn't set */ + + return _bt_readnextpage(scan, blkno, lastcurrblkno, dir, false); } /* - * _bt_readpage() -- Load data from current index page into so->currPos + * _bt_readpage() -- Load data from current index page into newbatch. * - * Caller must have pinned and read-locked so->currPos.buf; the buffer's state - * is not changed here. Also, currPos.moreLeft and moreRight must be valid; - * they are updated as appropriate. All other fields of so->currPos are - * initialized from scratch here. + * Caller must have pinned and read-locked newbatch.buf; the buffer's state is + * not changed here. Also, pos.moreLeft and moreRight must be valid; they are + * updated as appropriate. All other fields of newbatch are initialized from + * scratch here. * * We scan the current page starting at offnum and moving in the indicated - * direction. All items matching the scan keys are loaded into currPos.items. + * direction. All items matching the scan keys are saved in newbatch.items. * moreLeft or moreRight (as appropriate) is cleared if _bt_checkkeys reports * that there can be no more matching tuples in the current scan direction * (could just be for the current primitive index scan when scan has arrays). @@ -1644,8 +1618,8 @@ _bt_next(IndexScanDesc scan, ScanDirection dir) * Returns true if any matching items found on the page, false if none. */ static bool -_bt_readpage(IndexScanDesc scan, ScanDirection dir, OffsetNumber offnum, - bool firstpage) +_bt_readpage(IndexScanDesc scan, IndexScanBatch newbatch, ScanDirection dir, + OffsetNumber offnum, bool firstpage) { Relation rel = scan->indexRelation; BTScanOpaque so = (BTScanOpaque) scan->opaque; @@ -1656,37 +1630,35 @@ _bt_readpage(IndexScanDesc scan, ScanDirection dir, OffsetNumber offnum, BTReadPageState pstate; bool arrayKeys; int itemIndex, + tupleOffset = 0, indnatts; + BTScanPos pos = newbatch->pos; /* save the page/buffer block number, along with its sibling links */ - page = BufferGetPage(so->currPos.buf); + page = BufferGetPage(newbatch->buf); opaque = BTPageGetOpaque(page); - so->currPos.currPage = BufferGetBlockNumber(so->currPos.buf); - so->currPos.prevPage = opaque->btpo_prev; - so->currPos.nextPage = opaque->btpo_next; - /* delay setting so->currPos.lsn until _bt_drop_lock_and_maybe_pin */ - so->currPos.dir = dir; - so->currPos.nextTupleOffset = 0; + pos->currPage = BufferGetBlockNumber(newbatch->buf); + pos->prevPage = opaque->btpo_prev; + pos->nextPage = opaque->btpo_next; + pos->dir = dir; + + so->pos = pos; /* _bt_checkkeys needs this */ /* either moreRight or moreLeft should be set now (may be unset later) */ - Assert(ScanDirectionIsForward(dir) ? so->currPos.moreRight : - so->currPos.moreLeft); + Assert(ScanDirectionIsForward(dir) ? pos->moreRight : pos->moreLeft); Assert(!P_IGNORE(opaque)); - Assert(BTScanPosIsPinned(so->currPos)); Assert(!so->needPrimScan); if (scan->parallel_scan) { /* allow next/prev page to be read by other worker without delay */ if (ScanDirectionIsForward(dir)) - _bt_parallel_release(scan, so->currPos.nextPage, - so->currPos.currPage); + _bt_parallel_release(scan, pos->nextPage, pos->currPage); else - _bt_parallel_release(scan, so->currPos.prevPage, - so->currPos.currPage); + _bt_parallel_release(scan, pos->prevPage, pos->currPage); } - PredicateLockPage(rel, so->currPos.currPage, scan->xs_snapshot); + PredicateLockPage(rel, pos->currPage, scan->xs_snapshot); /* initialize local variables */ indnatts = IndexRelationGetNumberOfAttributes(rel); @@ -1724,11 +1696,10 @@ _bt_readpage(IndexScanDesc scan, ScanDirection dir, OffsetNumber offnum, !_bt_scanbehind_checkkeys(scan, dir, pstate.finaltup)) { /* Schedule another primitive index scan after all */ - so->currPos.moreRight = false; + pos->moreRight = false; so->needPrimScan = true; if (scan->parallel_scan) - _bt_parallel_primscan_schedule(scan, - so->currPos.currPage); + _bt_parallel_primscan_schedule(scan, pos->currPage); return false; } } @@ -1792,28 +1763,28 @@ _bt_readpage(IndexScanDesc scan, ScanDirection dir, OffsetNumber offnum, if (!BTreeTupleIsPosting(itup)) { /* Remember it */ - _bt_saveitem(so, itemIndex, offnum, itup); + _bt_saveitem(newbatch, itemIndex, offnum, itup, &tupleOffset); itemIndex++; } else { - int tupleOffset; + int baseOffset; /* * Set up state to return posting list, and remember first * TID */ - tupleOffset = - _bt_setuppostingitems(so, itemIndex, offnum, + baseOffset = + _bt_setuppostingitems(newbatch, itemIndex, offnum, BTreeTupleGetPostingN(itup, 0), - itup); + itup, &tupleOffset); itemIndex++; /* Remember additional TIDs */ for (int i = 1; i < BTreeTupleGetNPosting(itup); i++) { - _bt_savepostingitem(so, itemIndex, offnum, + _bt_savepostingitem(newbatch, itemIndex, offnum, BTreeTupleGetPostingN(itup, i), - tupleOffset); + baseOffset); itemIndex++; } } @@ -1853,12 +1824,11 @@ _bt_readpage(IndexScanDesc scan, ScanDirection dir, OffsetNumber offnum, } if (!pstate.continuescan) - so->currPos.moreRight = false; + pos->moreRight = false; Assert(itemIndex <= MaxTIDsPerBTreePage); - so->currPos.firstItem = 0; - so->currPos.lastItem = itemIndex - 1; - so->currPos.itemIndex = 0; + newbatch->firstItem = 0; + newbatch->lastItem = itemIndex - 1; } else { @@ -1875,11 +1845,10 @@ _bt_readpage(IndexScanDesc scan, ScanDirection dir, OffsetNumber offnum, !_bt_scanbehind_checkkeys(scan, dir, pstate.finaltup)) { /* Schedule another primitive index scan after all */ - so->currPos.moreLeft = false; + pos->moreLeft = false; so->needPrimScan = true; if (scan->parallel_scan) - _bt_parallel_primscan_schedule(scan, - so->currPos.currPage); + _bt_parallel_primscan_schedule(scan, pos->currPage); return false; } } @@ -1980,11 +1949,11 @@ _bt_readpage(IndexScanDesc scan, ScanDirection dir, OffsetNumber offnum, { /* Remember it */ itemIndex--; - _bt_saveitem(so, itemIndex, offnum, itup); + _bt_saveitem(newbatch, itemIndex, offnum, itup, &tupleOffset); } else { - int tupleOffset; + int baseOffset; /* * Set up state to return posting list, and remember first @@ -1997,17 +1966,17 @@ _bt_readpage(IndexScanDesc scan, ScanDirection dir, OffsetNumber offnum, * associated with the same posting list tuple. */ itemIndex--; - tupleOffset = - _bt_setuppostingitems(so, itemIndex, offnum, + baseOffset = + _bt_setuppostingitems(newbatch, itemIndex, offnum, BTreeTupleGetPostingN(itup, 0), - itup); + itup, &tupleOffset); /* Remember additional TIDs */ for (int i = 1; i < BTreeTupleGetNPosting(itup); i++) { itemIndex--; - _bt_savepostingitem(so, itemIndex, offnum, + _bt_savepostingitem(newbatch, itemIndex, offnum, BTreeTupleGetPostingN(itup, i), - tupleOffset); + baseOffset); } } } @@ -2023,12 +1992,11 @@ _bt_readpage(IndexScanDesc scan, ScanDirection dir, OffsetNumber offnum, * be found there */ if (!pstate.continuescan) - so->currPos.moreLeft = false; + pos->moreLeft = false; Assert(itemIndex >= 0); - so->currPos.firstItem = itemIndex; - so->currPos.lastItem = MaxTIDsPerBTreePage - 1; - so->currPos.itemIndex = MaxTIDsPerBTreePage - 1; + newbatch->firstItem = itemIndex; + newbatch->lastItem = MaxTIDsPerBTreePage - 1; } /* @@ -2045,202 +2013,96 @@ _bt_readpage(IndexScanDesc scan, ScanDirection dir, OffsetNumber offnum, */ Assert(!pstate.forcenonrequired); - return (so->currPos.firstItem <= so->currPos.lastItem); + return (newbatch->firstItem <= newbatch->lastItem); } -/* Save an index item into so->currPos.items[itemIndex] */ +/* Save an index item into newbatch.items[itemIndex] */ static void -_bt_saveitem(BTScanOpaque so, int itemIndex, - OffsetNumber offnum, IndexTuple itup) +_bt_saveitem(IndexScanBatch newbatch, int itemIndex, OffsetNumber offnum, + IndexTuple itup, int *tupleOffset) { - BTScanPosItem *currItem = &so->currPos.items[itemIndex]; - Assert(!BTreeTupleIsPivot(itup) && !BTreeTupleIsPosting(itup)); - currItem->heapTid = itup->t_tid; - currItem->indexOffset = offnum; - if (so->currTuples) + /* copy the populated part of the items array */ + newbatch->items[itemIndex].heapTid = itup->t_tid; + newbatch->items[itemIndex].indexOffset = offnum; + + if (newbatch->currTuples) { Size itupsz = IndexTupleSize(itup); - currItem->tupleOffset = so->currPos.nextTupleOffset; - memcpy(so->currTuples + so->currPos.nextTupleOffset, itup, itupsz); - so->currPos.nextTupleOffset += MAXALIGN(itupsz); + newbatch->items[itemIndex].tupleOffset = *tupleOffset; + memcpy(newbatch->currTuples + *tupleOffset, itup, itupsz); + *tupleOffset += MAXALIGN(itupsz); } } /* * Setup state to save TIDs/items from a single posting list tuple. * - * Saves an index item into so->currPos.items[itemIndex] for TID that is - * returned to scan first. Second or subsequent TIDs for posting list should - * be saved by calling _bt_savepostingitem(). + * Saves an index item into newbatch.items[itemIndex] for TID that is returned + * to scan first. Second or subsequent TIDs for posting list should be saved + * by calling _bt_savepostingitem(). * - * Returns an offset into tuple storage space that main tuple is stored at if - * needed. + * Returns baseOffset, an offset into tuple storage space that main tuple is + * stored at if needed. */ static int -_bt_setuppostingitems(BTScanOpaque so, int itemIndex, OffsetNumber offnum, - const ItemPointerData *heapTid, IndexTuple itup) +_bt_setuppostingitems(IndexScanBatch newbatch, int itemIndex, + OffsetNumber offnum, const ItemPointerData *heapTid, + IndexTuple itup, int *tupleOffset) { - BTScanPosItem *currItem = &so->currPos.items[itemIndex]; + IndexScanBatchPosItem *item = &newbatch->items[itemIndex]; Assert(BTreeTupleIsPosting(itup)); - currItem->heapTid = *heapTid; - currItem->indexOffset = offnum; - if (so->currTuples) + /* copy the populated part of the items array */ + item->heapTid = *heapTid; + item->indexOffset = offnum; + + if (newbatch->currTuples) { /* Save base IndexTuple (truncate posting list) */ IndexTuple base; Size itupsz = BTreeTupleGetPostingOffset(itup); itupsz = MAXALIGN(itupsz); - currItem->tupleOffset = so->currPos.nextTupleOffset; - base = (IndexTuple) (so->currTuples + so->currPos.nextTupleOffset); + item->tupleOffset = *tupleOffset; + base = (IndexTuple) (newbatch->currTuples + *tupleOffset); memcpy(base, itup, itupsz); /* Defensively reduce work area index tuple header size */ base->t_info &= ~INDEX_SIZE_MASK; base->t_info |= itupsz; - so->currPos.nextTupleOffset += itupsz; + *tupleOffset += itupsz; - return currItem->tupleOffset; + return item->tupleOffset; } return 0; } /* - * Save an index item into so->currPos.items[itemIndex] for current posting + * Save an index item into newbatch.items[itemIndex] for current posting * tuple. * * Assumes that _bt_setuppostingitems() has already been called for current - * posting list tuple. Caller passes its return value as tupleOffset. + * posting list tuple. Caller passes its return value as baseOffset. */ static inline void -_bt_savepostingitem(BTScanOpaque so, int itemIndex, OffsetNumber offnum, - ItemPointer heapTid, int tupleOffset) +_bt_savepostingitem(IndexScanBatch newbatch, int itemIndex, OffsetNumber offnum, + ItemPointer heapTid, int baseOffset) { - BTScanPosItem *currItem = &so->currPos.items[itemIndex]; + IndexScanBatchPosItem *item = &newbatch->items[itemIndex]; - currItem->heapTid = *heapTid; - currItem->indexOffset = offnum; + item->heapTid = *heapTid; + item->indexOffset = offnum; /* * Have index-only scans return the same base IndexTuple for every TID * that originates from the same posting list */ - if (so->currTuples) - currItem->tupleOffset = tupleOffset; -} - -/* - * Return the index item from so->currPos.items[so->currPos.itemIndex] to the - * index scan by setting the relevant fields in caller's index scan descriptor - */ -static inline void -_bt_returnitem(IndexScanDesc scan, BTScanOpaque so) -{ - BTScanPosItem *currItem = &so->currPos.items[so->currPos.itemIndex]; - - /* Most recent _bt_readpage must have succeeded */ - Assert(BTScanPosIsValid(so->currPos)); - Assert(so->currPos.itemIndex >= so->currPos.firstItem); - Assert(so->currPos.itemIndex <= so->currPos.lastItem); - - /* Return next item, per amgettuple contract */ - scan->xs_heaptid = currItem->heapTid; - if (so->currTuples) - scan->xs_itup = (IndexTuple) (so->currTuples + currItem->tupleOffset); -} - -/* - * _bt_steppage() -- Step to next page containing valid data for scan - * - * Wrapper on _bt_readnextpage that performs final steps for the current page. - * - * On entry, so->currPos must be valid. Its buffer will be pinned, though - * never locked. (Actually, when so->dropPin there won't even be a pin held, - * though so->currPos.currPage must still be set to a valid block number.) - */ -static bool -_bt_steppage(IndexScanDesc scan, ScanDirection dir) -{ - BTScanOpaque so = (BTScanOpaque) scan->opaque; - BlockNumber blkno, - lastcurrblkno; - - Assert(BTScanPosIsValid(so->currPos)); - - /* Before leaving current page, deal with any killed items */ - if (so->numKilled > 0) - _bt_killitems(scan); - - /* - * Before we modify currPos, make a copy of the page data if there was a - * mark position that needs it. - */ - if (so->markItemIndex >= 0) - { - /* bump pin on current buffer for assignment to mark buffer */ - if (BTScanPosIsPinned(so->currPos)) - IncrBufferRefCount(so->currPos.buf); - memcpy(&so->markPos, &so->currPos, - offsetof(BTScanPosData, items[1]) + - so->currPos.lastItem * sizeof(BTScanPosItem)); - if (so->markTuples) - memcpy(so->markTuples, so->currTuples, - so->currPos.nextTupleOffset); - so->markPos.itemIndex = so->markItemIndex; - so->markItemIndex = -1; - - /* - * If we're just about to start the next primitive index scan - * (possible with a scan that has arrays keys, and needs to skip to - * continue in the current scan direction), moreLeft/moreRight only - * indicate the end of the current primitive index scan. They must - * never be taken to indicate that the top-level index scan has ended - * (that would be wrong). - * - * We could handle this case by treating the current array keys as - * markPos state. But depending on the current array state like this - * would add complexity. Instead, we just unset markPos's copy of - * moreRight or moreLeft (whichever might be affected), while making - * btrestrpos reset the scan's arrays to their initial scan positions. - * In effect, btrestrpos leaves advancing the arrays up to the first - * _bt_readpage call (that takes place after it has restored markPos). - */ - if (so->needPrimScan) - { - if (ScanDirectionIsForward(so->currPos.dir)) - so->markPos.moreRight = true; - else - so->markPos.moreLeft = true; - } - - /* mark/restore not supported by parallel scans */ - Assert(!scan->parallel_scan); - } - - BTScanPosUnpinIfPinned(so->currPos); - - /* Walk to the next page with data */ - if (ScanDirectionIsForward(dir)) - blkno = so->currPos.nextPage; - else - blkno = so->currPos.prevPage; - lastcurrblkno = so->currPos.currPage; - - /* - * Cancel primitive index scans that were scheduled when the call to - * _bt_readpage for currPos happened to use the opposite direction to the - * one that we're stepping in now. (It's okay to leave the scan's array - * keys as-is, since the next _bt_readpage will advance them.) - */ - if (so->currPos.dir != dir) - so->needPrimScan = false; - - return _bt_readnextpage(scan, blkno, lastcurrblkno, dir, false); + if (newbatch->currTuples) + item->tupleOffset = baseOffset; } /* @@ -2252,73 +2114,96 @@ _bt_steppage(IndexScanDesc scan, ScanDirection dir) * to stop the scan on this page by calling _bt_checkkeys against the high * key. See _bt_readpage for full details. * - * On entry, so->currPos must be pinned and locked (so offnum stays valid). + * On entry, firstbatch must be pinned and locked (so offnum stays valid). * Parallel scan callers must have seized the scan before calling here. * - * On exit, we'll have updated so->currPos and retained locks and pins + * On exit, we'll have updated firstbatch and retained locks and pins * according to the same rules as those laid out for _bt_readnextpage exit. - * Like _bt_readnextpage, our return value indicates if there are any matching - * records in the given direction. * * We always release the scan for a parallel scan caller, regardless of * success or failure; we'll call _bt_parallel_release as soon as possible. */ -static bool -_bt_readfirstpage(IndexScanDesc scan, OffsetNumber offnum, ScanDirection dir) +static IndexScanBatch +_bt_readfirstpage(IndexScanDesc scan, IndexScanBatch firstbatch, + OffsetNumber offnum, ScanDirection dir) { BTScanOpaque so = (BTScanOpaque) scan->opaque; + Relation rel = scan->indexRelation; + BlockNumber blkno, + lastcurrblkno; + BTScanPos firstpos = firstbatch->pos; - so->numKilled = 0; /* just paranoia */ - so->markItemIndex = -1; /* ditto */ - - /* Initialize so->currPos for the first page (page in so->currPos.buf) */ + /* Initialize firstbatch's position for the first page */ if (so->needPrimScan) { Assert(so->numArrayKeys); - so->currPos.moreLeft = true; - so->currPos.moreRight = true; + firstpos->moreLeft = true; + firstpos->moreRight = true; so->needPrimScan = false; } else if (ScanDirectionIsForward(dir)) { - so->currPos.moreLeft = false; - so->currPos.moreRight = true; + firstpos->moreLeft = false; + firstpos->moreRight = true; } else { - so->currPos.moreLeft = true; - so->currPos.moreRight = false; + firstpos->moreLeft = true; + firstpos->moreRight = false; } /* * Attempt to load matching tuples from the first page. * - * Note that _bt_readpage will finish initializing the so->currPos fields. + * Note that _bt_readpage will finish initializing the firstbatch fields. * _bt_readpage also releases parallel scan (even when it returns false). */ - if (_bt_readpage(scan, dir, offnum, true)) + if (_bt_readpage(scan, firstbatch, dir, offnum, true)) { - Relation rel = scan->indexRelation; + /* _bt_readpage succeeded */ + index_batch_unlock(rel, scan->batchState && scan->batchState->dropPin, + firstbatch); + return firstbatch; + } + + /* There's no actually-matching data on the page in firstbatch->buf */ + _bt_relbuf(rel, firstbatch->buf); + firstbatch->buf = InvalidBuffer; + /* Walk to the next page with data */ + if (ScanDirectionIsForward(dir)) + blkno = firstpos->nextPage; + else + blkno = firstpos->prevPage; + lastcurrblkno = firstpos->currPage; + + Assert(firstpos->dir == dir); + + /* firstbatch will never be returned to scan, so free it outselves */ + pfree(firstbatch); + + if (blkno == P_NONE || + (ScanDirectionIsForward(dir) ? + !firstpos->moreRight : !firstpos->moreLeft)) + { /* - * _bt_readpage succeeded. Drop the lock (and maybe the pin) on - * so->currPos.buf in preparation for btgettuple returning tuples. + * firstbatch _bt_readpage call ended scan in this direction (though + * if so->needPrimScan was set the scan will continue in _bt_first) */ - Assert(BTScanPosIsPinned(so->currPos)); - _bt_drop_lock_and_maybe_pin(rel, so); - return true; + pfree(firstpos); + _bt_parallel_done(scan); + return NULL; } - /* There's no actually-matching data on the page in so->currPos.buf */ - _bt_unlockbuf(scan->indexRelation, so->currPos.buf); + pfree(firstpos); - /* Call _bt_readnextpage using its _bt_steppage wrapper function */ - if (!_bt_steppage(scan, dir)) - return false; + /* parallel scan must seize the scan to get next blkno */ + if (scan->parallel_scan != NULL && + !_bt_parallel_seize(scan, &blkno, &lastcurrblkno, false)) + return NULL; /* done iff so->needPrimScan wasn't set */ - /* _bt_readpage for a later page (now in so->currPos) succeeded */ - return true; + return _bt_readnextpage(scan, blkno, lastcurrblkno, dir, false); } /* @@ -2328,102 +2213,70 @@ _bt_readfirstpage(IndexScanDesc scan, OffsetNumber offnum, ScanDirection dir) * previously-saved right link or left link. lastcurrblkno is the page that * was current at the point where the blkno link was saved, which we use to * reason about concurrent page splits/page deletions during backwards scans. - * In the common case where seized=false, blkno is either so->currPos.nextPage - * or so->currPos.prevPage, and lastcurrblkno is so->currPos.currPage. + * blkno is the prior scan position's nextPage or prevPage (depending on scan + * direction), and lastcurrblkno is the prior position's currPage. * - * On entry, so->currPos shouldn't be locked by caller. so->currPos.buf must - * be InvalidBuffer/unpinned as needed by caller (note that lastcurrblkno - * won't need to be read again in almost all cases). Parallel scan callers - * that seized the scan before calling here should pass seized=true; such a - * caller's blkno and lastcurrblkno arguments come from the seized scan. - * seized=false callers just pass us the blkno/lastcurrblkno taken from their - * so->currPos, which (along with so->currPos itself) can be used to end the - * scan. A seized=false caller's blkno can never be assumed to be the page - * that must be read next during a parallel scan, though. We must figure that - * part out for ourselves by seizing the scan (the correct page to read might - * already be beyond the seized=false caller's blkno during a parallel scan, - * unless blkno/so->currPos.nextPage/so->currPos.prevPage is already P_NONE, - * or unless so->currPos.moreRight/so->currPos.moreLeft is already unset). + * On entry, no page should be locked by caller. * - * On success exit, so->currPos is updated to contain data from the next - * interesting page, and we return true. We hold a pin on the buffer on - * success exit (except during so->dropPin index scans, when we drop the pin - * eagerly to avoid blocking VACUUM). + * On success exit, returns scan batch containing data from the next + * interesting page. We hold a pin on the buffer on success exit (except + * during dropPin plain index scans, when we drop the pin eagerly to avoid + * blocking VACUUM). If there are no more matching records in the given + * direction, we just return NULL. * - * If there are no more matching records in the given direction, we invalidate - * so->currPos (while ensuring it retains no locks or pins), and return false. - * - * We always release the scan for a parallel scan caller, regardless of - * success or failure; we'll call _bt_parallel_release as soon as possible. + * Parallel scan callers must seize the scan before calling here. blkno and + * lastcurrblkno should come from the seized scan. We'll release the scan as + * soon as possible. */ -static bool +static IndexScanBatch _bt_readnextpage(IndexScanDesc scan, BlockNumber blkno, - BlockNumber lastcurrblkno, ScanDirection dir, bool seized) + BlockNumber lastcurrblkno, ScanDirection dir, bool firstpage) { Relation rel = scan->indexRelation; - BTScanOpaque so = (BTScanOpaque) scan->opaque; + IndexScanBatch newbatch; + BTScanPos newpos; - Assert(so->currPos.currPage == lastcurrblkno || seized); - Assert(!(blkno == P_NONE && seized)); - Assert(!BTScanPosIsPinned(so->currPos)); + /* Allocate space for next batch */ + newbatch = index_batch_alloc(MaxTIDsPerBTreePage, scan->xs_want_itup); + newbatch->pos = palloc(sizeof(BTScanPosData)); + newpos = newbatch->pos; /* - * Remember that the scan already read lastcurrblkno, a page to the left - * of blkno (or remember reading a page to the right, for backwards scans) + * pos is the first valid page to the right (or to the left) of + * lastcurrblkno. Also provisionally assume that there'll be another page + * we'll need to the right (or to the left) ahead of _bt_readpage call. */ - if (ScanDirectionIsForward(dir)) - so->currPos.moreLeft = true; - else - so->currPos.moreRight = true; + newpos->moreLeft = true; + newpos->moreRight = true; for (;;) { Page page; BTPageOpaque opaque; - if (blkno == P_NONE || - (ScanDirectionIsForward(dir) ? - !so->currPos.moreRight : !so->currPos.moreLeft)) - { - /* most recent _bt_readpage call (for lastcurrblkno) ended scan */ - Assert(so->currPos.currPage == lastcurrblkno && !seized); - BTScanPosInvalidate(so->currPos); - _bt_parallel_done(scan); /* iff !so->needPrimScan */ - return false; - } - - Assert(!so->needPrimScan); - - /* parallel scan must never actually visit so->currPos blkno */ - if (!seized && scan->parallel_scan != NULL && - !_bt_parallel_seize(scan, &blkno, &lastcurrblkno, false)) - { - /* whole scan is now done (or another primitive scan required) */ - BTScanPosInvalidate(so->currPos); - return false; - } + Assert(!((BTScanOpaque) scan->opaque)->needPrimScan); + Assert(blkno != P_NONE && lastcurrblkno != P_NONE); if (ScanDirectionIsForward(dir)) { /* read blkno, but check for interrupts first */ CHECK_FOR_INTERRUPTS(); - so->currPos.buf = _bt_getbuf(rel, blkno, BT_READ); + newbatch->buf = _bt_getbuf(rel, blkno, BT_READ); } else { /* read blkno, avoiding race (also checks for interrupts) */ - so->currPos.buf = _bt_lock_and_validate_left(rel, &blkno, - lastcurrblkno); - if (so->currPos.buf == InvalidBuffer) + newbatch->buf = _bt_lock_and_validate_left(rel, &blkno, + lastcurrblkno); + if (newbatch->buf == InvalidBuffer) { /* must have been a concurrent deletion of leftmost page */ - BTScanPosInvalidate(so->currPos); _bt_parallel_done(scan); - return false; + return NULL; } } - page = BufferGetPage(so->currPos.buf); + page = BufferGetPage(newbatch->buf); opaque = BTPageGetOpaque(page); lastcurrblkno = blkno; if (likely(!P_IGNORE(opaque))) @@ -2431,17 +2284,17 @@ _bt_readnextpage(IndexScanDesc scan, BlockNumber blkno, /* see if there are any matches on this page */ if (ScanDirectionIsForward(dir)) { - /* note that this will clear moreRight if we can stop */ - if (_bt_readpage(scan, dir, P_FIRSTDATAKEY(opaque), seized)) + if (_bt_readpage(scan, newbatch, dir, + P_FIRSTDATAKEY(opaque), firstpage)) break; - blkno = so->currPos.nextPage; + blkno = newpos->nextPage; } else { - /* note that this will clear moreLeft if we can stop */ - if (_bt_readpage(scan, dir, PageGetMaxOffsetNumber(page), seized)) + if (_bt_readpage(scan, newbatch, dir, + PageGetMaxOffsetNumber(page), firstpage)) break; - blkno = so->currPos.prevPage; + blkno = newpos->prevPage; } } else @@ -2456,19 +2309,36 @@ _bt_readnextpage(IndexScanDesc scan, BlockNumber blkno, } /* no matching tuples on this page */ - _bt_relbuf(rel, so->currPos.buf); - seized = false; /* released by _bt_readpage (or by us) */ + _bt_relbuf(rel, newbatch->buf); + newbatch->buf = InvalidBuffer; + + /* Continue the scan in this direction? */ + if (blkno == P_NONE || + (ScanDirectionIsForward(dir) ? + !newpos->moreRight : !newpos->moreLeft)) + { + /* + * blkno _bt_readpage call ended scan in this direction (though if + * so->needPrimScan was set the scan will continue in _bt_first) + */ + _bt_parallel_done(scan); + return NULL; + } + + /* parallel scan must seize the scan to get next blkno */ + if (scan->parallel_scan != NULL && + !_bt_parallel_seize(scan, &blkno, &lastcurrblkno, false)) + return NULL; /* done iff so->needPrimScan wasn't set */ + + firstpage = false; /* next page cannot be first */ } - /* - * _bt_readpage succeeded. Drop the lock (and maybe the pin) on - * so->currPos.buf in preparation for btgettuple returning tuples. - */ - Assert(so->currPos.currPage == blkno); - Assert(BTScanPosIsPinned(so->currPos)); - _bt_drop_lock_and_maybe_pin(rel, so); + /* _bt_readpage succeeded */ + Assert(newpos->currPage == blkno); + index_batch_unlock(rel, scan->batchState && scan->batchState->dropPin, + newbatch); - return true; + return newbatch; } /* @@ -2691,25 +2561,23 @@ _bt_get_endpoint(Relation rel, uint32 level, bool rightmost) * Parallel scan callers must have seized the scan before calling here. * Exit conditions are the same as for _bt_first(). */ -static bool -_bt_endpoint(IndexScanDesc scan, ScanDirection dir) +static IndexScanBatch +_bt_endpoint(IndexScanDesc scan, ScanDirection dir, IndexScanBatch firstbatch) { Relation rel = scan->indexRelation; - BTScanOpaque so = (BTScanOpaque) scan->opaque; Page page; BTPageOpaque opaque; OffsetNumber start; - Assert(!BTScanPosIsValid(so->currPos)); - Assert(!so->needPrimScan); + Assert(!((BTScanOpaque) scan->opaque)->needPrimScan); /* * Scan down to the leftmost or rightmost leaf page. This is a simplified * version of _bt_search(). */ - so->currPos.buf = _bt_get_endpoint(rel, 0, ScanDirectionIsBackward(dir)); + firstbatch->buf = _bt_get_endpoint(rel, 0, ScanDirectionIsBackward(dir)); - if (!BufferIsValid(so->currPos.buf)) + if (!BufferIsValid(firstbatch->buf)) { /* * Empty index. Lock the whole relation, as nothing finer to lock @@ -2720,7 +2588,7 @@ _bt_endpoint(IndexScanDesc scan, ScanDirection dir) return false; } - page = BufferGetPage(so->currPos.buf); + page = BufferGetPage(firstbatch->buf); opaque = BTPageGetOpaque(page); Assert(P_ISLEAF(opaque)); @@ -2746,9 +2614,5 @@ _bt_endpoint(IndexScanDesc scan, ScanDirection dir) /* * Now load data from the first page of the scan. */ - if (!_bt_readfirstpage(scan, start, dir)) - return false; - - _bt_returnitem(scan, so); - return true; + return _bt_readfirstpage(scan, firstbatch, start, dir); } diff --git a/src/backend/access/nbtree/nbtutils.c b/src/backend/access/nbtree/nbtutils.c index ab0f98b0287c..9872de87a2d9 100644 --- a/src/backend/access/nbtree/nbtutils.c +++ b/src/backend/access/nbtree/nbtutils.c @@ -1022,14 +1022,6 @@ _bt_advance_array_keys_increment(IndexScanDesc scan, ScanDirection dir, * Restore the array keys to the state they were in immediately before we * were called. This ensures that the arrays only ever ratchet in the * current scan direction. - * - * Without this, scans could overlook matching tuples when the scan - * direction gets reversed just before btgettuple runs out of items to - * return, but just after _bt_readpage prepares all the items from the - * scan's final page in so->currPos. When we're on the final page it is - * typical for so->currPos to get invalidated once btgettuple finally - * returns false, which'll effectively invalidate the scan's array keys. - * That hasn't happened yet, though -- and in general it may never happen. */ _bt_start_array_keys(scan, -dir); @@ -1396,7 +1388,7 @@ _bt_advance_array_keys(IndexScanDesc scan, BTReadPageState *pstate, { BTScanOpaque so = (BTScanOpaque) scan->opaque; Relation rel = scan->indexRelation; - ScanDirection dir = so->currPos.dir; + ScanDirection dir = so->pos->dir; int arrayidx = 0; bool beyond_end_advance = false, skip_array_advanced = false, @@ -2033,13 +2025,13 @@ _bt_advance_array_keys(IndexScanDesc scan, BTReadPageState *pstate, * Note: We make a soft assumption that the current scan direction will * also be used within _bt_next, when it is asked to step off this page. * It is up to _bt_next to cancel this scheduled primitive index scan - * whenever it steps to a page in the direction opposite currPos.dir. + * whenever it steps to a page in the direction opposite pos->dir. */ pstate->continuescan = false; /* Tell _bt_readpage we're done... */ so->needPrimScan = true; /* ...but call _bt_first again */ if (scan->parallel_scan) - _bt_parallel_primscan_schedule(scan, so->currPos.currPage); + _bt_parallel_primscan_schedule(scan, so->pos->currPage); /* Caller's tuple doesn't match the new qual */ return false; @@ -2152,7 +2144,7 @@ _bt_checkkeys(IndexScanDesc scan, BTReadPageState *pstate, bool arrayKeys, { TupleDesc tupdesc = RelationGetDescr(scan->indexRelation); BTScanOpaque so = (BTScanOpaque) scan->opaque; - ScanDirection dir = so->currPos.dir; + ScanDirection dir = so->pos->dir; int ikey = pstate->startikey; bool res; @@ -3302,7 +3294,7 @@ _bt_checkkeys_look_ahead(IndexScanDesc scan, BTReadPageState *pstate, int tupnatts, TupleDesc tupdesc) { BTScanOpaque so = (BTScanOpaque) scan->opaque; - ScanDirection dir = so->currPos.dir; + ScanDirection dir = so->pos->dir; OffsetNumber aheadoffnum; IndexTuple ahead; @@ -3376,69 +3368,67 @@ _bt_checkkeys_look_ahead(IndexScanDesc scan, BTReadPageState *pstate, * current page and killed tuples thereon (generally, this should only be * called if so->numKilled > 0). * - * Caller should not have a lock on the so->currPos page, but must hold a - * buffer pin when !so->dropPin. When we return, it still won't be locked. - * It'll continue to hold whatever pins were held before calling here. + * Caller should not have a lock on the batch position's page, but must hold a + * buffer pin when !dropPin. When we return, it still won't be locked. It'll + * continue to hold whatever pins were held before calling here. * * We match items by heap TID before assuming they are the right ones to set * LP_DEAD. If the scan is one that holds a buffer pin on the target page * continuously from initially reading the items until applying this function - * (if it is a !so->dropPin scan), VACUUM cannot have deleted any items on the + * (if it is a !dropPin scan), VACUUM cannot have deleted any items on the * page, so the page's TIDs can't have been recycled by now. There's no risk * that we'll confuse a new index tuple that happens to use a recycled TID * with a now-removed tuple with the same TID (that used to be on this same * page). We can't rely on that during scans that drop buffer pins eagerly - * (so->dropPin scans), though, so we must condition setting LP_DEAD bits on + * (i.e. dropPin scans), though, so we must condition setting LP_DEAD bits on * the page LSN having not changed since back when _bt_readpage saw the page. * We totally give up on setting LP_DEAD bits when the page LSN changed. * - * We give up much less often during !so->dropPin scans, but it still happens. + * We tend to give up less often during !dropPin scans, but it still happens. * We cope with cases where items have moved right due to insertions. If an * item has moved off the current page due to a split, we'll fail to find it * and just give up on it. */ void -_bt_killitems(IndexScanDesc scan) +_bt_killitems(IndexScanDesc scan, IndexScanBatch batch) { Relation rel = scan->indexRelation; - BTScanOpaque so = (BTScanOpaque) scan->opaque; + BTScanPos pos = (BTScanPos) batch->pos; Page page; BTPageOpaque opaque; OffsetNumber minoff; OffsetNumber maxoff; - int numKilled = so->numKilled; + int numKilled = batch->numKilled; bool killedsomething = false; Buffer buf; Assert(numKilled > 0); - Assert(BTScanPosIsValid(so->currPos)); + Assert(BTScanPosIsValid(*pos)); Assert(scan->heapRelation != NULL); /* can't be a bitmap index scan */ - /* Always invalidate so->killedItems[] before leaving so->currPos */ - so->numKilled = 0; + /* Always invalidate batch->killedItems[] before freeing batch */ + batch->numKilled = 0; - if (!so->dropPin) + if (!scan->batchState->dropPin) { /* * We have held the pin on this page since we read the index tuples, * so all we need to do is lock it. The pin will have prevented * concurrent VACUUMs from recycling any of the TIDs on the page. */ - Assert(BTScanPosIsPinned(so->currPos)); - buf = so->currPos.buf; + buf = batch->buf; _bt_lockbuf(rel, buf, BT_READ); } else { XLogRecPtr latestlsn; - Assert(!BTScanPosIsPinned(so->currPos)); Assert(RelationNeedsWAL(rel)); - buf = _bt_getbuf(rel, so->currPos.currPage, BT_READ); + buf = _bt_getbuf(rel, pos->currPage, BT_READ); latestlsn = BufferGetLSNAtomic(buf); - Assert(so->currPos.lsn <= latestlsn); - if (so->currPos.lsn != latestlsn) + Assert(batch->lsn <= latestlsn); + if (batch->lsn != latestlsn) { /* Modified, give up on hinting */ _bt_relbuf(rel, buf); @@ -3455,12 +3445,11 @@ _bt_killitems(IndexScanDesc scan) for (int i = 0; i < numKilled; i++) { - int itemIndex = so->killedItems[i]; - BTScanPosItem *kitem = &so->currPos.items[itemIndex]; + int itemIndex = batch->killedItems[i]; + IndexScanBatchPosItem *kitem = &batch->items[itemIndex]; OffsetNumber offnum = kitem->indexOffset; - Assert(itemIndex >= so->currPos.firstItem && - itemIndex <= so->currPos.lastItem); + Assert(itemIndex >= batch->firstItem && itemIndex <= batch->lastItem); if (offnum < minoff) continue; /* pure paranoia */ while (offnum <= maxoff) @@ -3485,7 +3474,7 @@ _bt_killitems(IndexScanDesc scan) * correctness. * * Note that the page may have been modified in almost any way - * since we first read it (in the !so->dropPin case), so it's + * since we first read it (in the !dropPin case), so it's * possible that this posting list tuple wasn't a posting list * tuple when we first encountered its heap TIDs. */ @@ -3501,7 +3490,8 @@ _bt_killitems(IndexScanDesc scan) * though only in the common case where the page can't * have been concurrently modified */ - Assert(kitem->indexOffset == offnum || !so->dropPin); + Assert(kitem->indexOffset == offnum || + !scan->batchState->dropPin); /* * Read-ahead to later kitems here. @@ -3518,7 +3508,7 @@ _bt_killitems(IndexScanDesc scan) * correctly -- posting tuple still gets killed). */ if (pi < numKilled) - kitem = &so->currPos.items[so->killedItems[pi++]]; + kitem = &batch->items[batch->killedItems[pi++]]; } /* @@ -3568,7 +3558,7 @@ _bt_killitems(IndexScanDesc scan) MarkBufferDirtyHint(buf, true); } - if (!so->dropPin) + if (!scan->batchState->dropPin) _bt_unlockbuf(rel, buf); else _bt_relbuf(rel, buf); diff --git a/src/backend/access/spgist/spgutils.c b/src/backend/access/spgist/spgutils.c index 87c31da71a52..9d66d26ddb50 100644 --- a/src/backend/access/spgist/spgutils.c +++ b/src/backend/access/spgist/spgutils.c @@ -88,10 +88,11 @@ spghandler(PG_FUNCTION_ARGS) amroutine->ambeginscan = spgbeginscan; amroutine->amrescan = spgrescan; amroutine->amgettuple = spggettuple; + amroutine->amgetbatch = NULL; + amroutine->amfreebatch = NULL; amroutine->amgetbitmap = spggetbitmap; amroutine->amendscan = spgendscan; - amroutine->ammarkpos = NULL; - amroutine->amrestrpos = NULL; + amroutine->amposreset = NULL; amroutine->amestimateparallelscan = NULL; amroutine->aminitparallelscan = NULL; amroutine->amparallelrescan = NULL; diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c index 5712fac36971..6d895e4ffce7 100644 --- a/src/backend/commands/indexcmds.c +++ b/src/backend/commands/indexcmds.c @@ -880,7 +880,7 @@ DefineIndex(Oid tableId, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("access method \"%s\" does not support multicolumn indexes", accessMethodName))); - if (exclusion && amRoutine->amgettuple == NULL) + if (exclusion && amRoutine->amgettuple == NULL && amRoutine->amgetbatch == NULL) ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("access method \"%s\" does not support exclusion constraints", diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c index 1d0e8ad57b4a..ac337d9000f1 100644 --- a/src/backend/executor/execAmi.c +++ b/src/backend/executor/execAmi.c @@ -428,7 +428,7 @@ ExecSupportsMarkRestore(Path *pathnode) case T_IndexOnlyScan: /* - * Not all index types support mark/restore. + * Not all index types support restoring a mark */ return castNode(IndexPath, pathnode)->indexinfo->amcanmarkpos; diff --git a/src/backend/executor/nodeIndexonlyscan.c b/src/backend/executor/nodeIndexonlyscan.c index f464cca9507a..5e7bafe07a34 100644 --- a/src/backend/executor/nodeIndexonlyscan.c +++ b/src/backend/executor/nodeIndexonlyscan.c @@ -49,7 +49,13 @@ static TupleTableSlot *IndexOnlyNext(IndexOnlyScanState *node); static void StoreIndexTuple(IndexOnlyScanState *node, TupleTableSlot *slot, IndexTuple itup, TupleDesc itupdesc); +static bool ios_prefetch_block(IndexScanDesc scan, void *arg, + IndexScanBatchPos *pos); +/* values stored in ios_prefetch_block in the batch cache */ +#define IOS_UNKNOWN_VISIBILITY 0 /* default value */ +#define IOS_ALL_VISIBLE 1 +#define IOS_NOT_ALL_VISIBLE 2 /* ---------------------------------------------------------------- * IndexOnlyNext @@ -103,6 +109,17 @@ IndexOnlyNext(IndexOnlyScanState *node) node->ioss_ScanDesc->xs_want_itup = true; node->ioss_VMBuffer = InvalidBuffer; + /* + * Set the prefetch callback info, if the scan has batching enabled + * (we only know what after index_beginscan, which also checks which + * callbacks are defined for the AM. + */ + if (scandesc->batchState != NULL) + { + scandesc->batchState->prefetch = ios_prefetch_block; + scandesc->batchState->prefetchArg = (void *) node; + } + /* * If no run-time keys to calculate or they are ready, go ahead and * pass the scankeys to the index AM. @@ -120,10 +137,42 @@ IndexOnlyNext(IndexOnlyScanState *node) */ while ((tid = index_getnext_tid(scandesc, direction)) != NULL) { + bool all_visible; bool tuple_from_heap = false; CHECK_FOR_INTERRUPTS(); + /* + * Without batching, inspect the VM directly. With batching, we need + * to retrieve the visibility information seen by the read_stream + * callback (or rather by ios_prefetch_block), otherwise the + * read_stream might get out of sync (if the VM got updated since + * then). + */ + if (scandesc->batchState == NULL) + { + all_visible = VM_ALL_VISIBLE(scandesc->heapRelation, + ItemPointerGetBlockNumber(tid), + &node->ioss_VMBuffer); + } + else + { + /* + * Reuse the previously determined page visibility info, or + * calculate it now. If we decided not to prefetch the block, the + * page had to be all-visible at that point. The VM bit might have + * changed since then, but the tuple visibility could not have. + * + * XXX It's a bit weird we use the visibility to decide if we + * should skip prefetching the block, and then deduce the + * visibility from that (even if it matches pretty clearly). But + * maybe we could/should have a more direct way to read the + * private state? + */ + all_visible = !ios_prefetch_block(scandesc, node, + &scandesc->batchState->readPos); + } + /* * We can skip the heap fetch if the TID references a heap page on * which all tuples are known visible to everybody. In any case, @@ -158,9 +207,7 @@ IndexOnlyNext(IndexOnlyScanState *node) * It's worth going through this complexity to avoid needing to lock * the VM buffer, which could cause significant contention. */ - if (!VM_ALL_VISIBLE(scandesc->heapRelation, - ItemPointerGetBlockNumber(tid), - &node->ioss_VMBuffer)) + if (!all_visible) { /* * Rats, we have to visit the heap to check visibility. @@ -889,3 +936,51 @@ ExecIndexOnlyScanRetrieveInstrumentation(IndexOnlyScanState *node) node->ioss_SharedInfo = palloc(size); memcpy(node->ioss_SharedInfo, SharedInfo, size); } + +/* FIXME duplicate from indexam.c */ +#define INDEX_SCAN_BATCH(scan, idx) \ + ((scan)->batchState->batches[(idx) % (scan)->batchState->maxBatches]) + +/* + * ios_prefetch_block + * Callback to only prefetch blocks that are not all-visible. + * + * We don't want to inspect the visibility map repeatedly, so the result of + * VM_ALL_VISIBLE is stored in the batch private data. The values are set + * to 0 by default, so we use two constants to remember if all-visible or + * not all-visible. + * + * However, this is not merely a question of performance. The VM may get + * modified during the scan, and we need to make sure the two places (the + * read_next callback and the index_fetch_heap here) make the same decision, + * otherwise we might get out of sync with the stream. For example, the + * callback might find a page is all-visible (and skips reading the block), + * and then someone might update the page, resetting the VM bit. If this + * place attempts to read the page from the stream, it'll fail because it + * will probably receive an entirely different page. + */ +static bool +ios_prefetch_block(IndexScanDesc scan, void *arg, IndexScanBatchPos *pos) +{ + IndexOnlyScanState *node = (IndexOnlyScanState *) arg; + IndexScanBatch batch = INDEX_SCAN_BATCH(scan, pos->batch); + + if (batch->itemsvisibility == NULL) + batch->itemsvisibility = palloc0(sizeof(char) * (batch->lastItem + 1)); + + if (batch->itemsvisibility[pos->index] == IOS_UNKNOWN_VISIBILITY) + { + bool all_visible; + ItemPointer tid = &batch->items[pos->index].heapTid; + + all_visible = VM_ALL_VISIBLE(scan->heapRelation, + ItemPointerGetBlockNumber(tid), + &node->ioss_VMBuffer); + + batch->itemsvisibility[pos->index] = + all_visible ? IOS_ALL_VISIBLE : IOS_NOT_ALL_VISIBLE; + } + + /* prefetch only blocks that are not all-visible */ + return (batch->itemsvisibility[pos->index] == IOS_NOT_ALL_VISIBLE); +} diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c index 8335cf5b5c5a..5dd89ed096a2 100644 --- a/src/backend/optimizer/path/costsize.c +++ b/src/backend/optimizer/path/costsize.c @@ -144,6 +144,7 @@ int max_parallel_workers_per_gather = 2; bool enable_seqscan = true; bool enable_indexscan = true; +bool enable_indexscan_prefetch = true; bool enable_indexonlyscan = true; bool enable_bitmapscan = true; bool enable_tidscan = true; diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c index d950bd930021..3e1ee438d186 100644 --- a/src/backend/optimizer/util/plancat.c +++ b/src/backend/optimizer/util/plancat.c @@ -313,11 +313,11 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent, info->amsearcharray = amroutine->amsearcharray; info->amsearchnulls = amroutine->amsearchnulls; info->amcanparallel = amroutine->amcanparallel; - info->amhasgettuple = (amroutine->amgettuple != NULL); + info->amhasgettuple = (amroutine->amgettuple != NULL || + amroutine->amgetbatch != NULL); info->amhasgetbitmap = amroutine->amgetbitmap != NULL && relation->rd_tableam->scan_bitmap_next_tuple != NULL; - info->amcanmarkpos = (amroutine->ammarkpos != NULL && - amroutine->amrestrpos != NULL); + info->amcanmarkpos = amroutine->amposreset != NULL; info->amcostestimate = amroutine->amcostestimate; Assert(info->amcostestimate != NULL); diff --git a/src/backend/replication/logical/relation.c b/src/backend/replication/logical/relation.c index 745fd3bab640..de6c66a15241 100644 --- a/src/backend/replication/logical/relation.c +++ b/src/backend/replication/logical/relation.c @@ -888,7 +888,8 @@ IsIndexUsableForReplicaIdentityFull(Relation idxrel, AttrMap *attrmap) * The given index access method must implement "amgettuple", which will * be used later to fetch the tuples. See RelationFindReplTupleByIndex(). */ - if (GetIndexAmRoutineByAmId(idxrel->rd_rel->relam, false)->amgettuple == NULL) + if (GetIndexAmRoutineByAmId(idxrel->rd_rel->relam, false)->amgettuple == NULL && + GetIndexAmRoutineByAmId(idxrel->rd_rel->relam, false)->amgetbatch == NULL) return false; return true; diff --git a/src/backend/storage/aio/read_stream.c b/src/backend/storage/aio/read_stream.c index 031fde9f4cbe..e34e60060604 100644 --- a/src/backend/storage/aio/read_stream.c +++ b/src/backend/storage/aio/read_stream.c @@ -99,6 +99,7 @@ struct ReadStream int16 forwarded_buffers; int16 pinned_buffers; int16 distance; + int16 distance_old; int16 initialized_buffers; int read_buffers_flags; bool sync_mode; /* using io_method=sync */ @@ -464,6 +465,7 @@ read_stream_look_ahead(ReadStream *stream) if (blocknum == InvalidBlockNumber) { /* End of stream. */ + stream->distance_old = stream->distance; stream->distance = 0; break; } @@ -862,6 +864,7 @@ read_stream_next_buffer(ReadStream *stream, void **per_buffer_data) else { /* No more blocks, end of stream. */ + stream->distance_old = stream->distance; stream->distance = 0; stream->oldest_buffer_index = stream->next_buffer_index; stream->pinned_buffers = 0; @@ -1046,6 +1049,9 @@ read_stream_reset(ReadStream *stream) int16 index; Buffer buffer; + /* remember the old distance (if we reset before end of the stream) */ + stream->distance_old = Max(stream->distance, stream->distance_old); + /* Stop looking ahead. */ stream->distance = 0; @@ -1078,8 +1084,12 @@ read_stream_reset(ReadStream *stream) Assert(stream->pinned_buffers == 0); Assert(stream->ios_in_progress == 0); - /* Start off assuming data is cached. */ - stream->distance = 1; + /* + * Restore the old distance, if we have one. Otherwise start assuming data + * is cached. + */ + stream->distance = Max(1, stream->distance_old); + stream->distance_old = 0; } /* diff --git a/src/backend/utils/adt/amutils.c b/src/backend/utils/adt/amutils.c index 0af26d6acfab..1ebe0a76af43 100644 --- a/src/backend/utils/adt/amutils.c +++ b/src/backend/utils/adt/amutils.c @@ -363,7 +363,7 @@ indexam_property(FunctionCallInfo fcinfo, PG_RETURN_BOOL(routine->amclusterable); case AMPROP_INDEX_SCAN: - PG_RETURN_BOOL(routine->amgettuple ? true : false); + PG_RETURN_BOOL(routine->amgettuple || routine->amgetbatch ? true : false); case AMPROP_BITMAP_SCAN: PG_RETURN_BOOL(routine->amgetbitmap ? true : false); @@ -392,7 +392,7 @@ indexam_property(FunctionCallInfo fcinfo, PG_RETURN_BOOL(routine->amcanmulticol); case AMPROP_CAN_EXCLUDE: - PG_RETURN_BOOL(routine->amgettuple ? true : false); + PG_RETURN_BOOL(routine->amgettuple || routine->amgetbatch ? true : false); case AMPROP_CAN_INCLUDE: PG_RETURN_BOOL(routine->amcaninclude); diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c index cb23ad527826..652bb4c537d5 100644 --- a/src/backend/utils/adt/selfuncs.c +++ b/src/backend/utils/adt/selfuncs.c @@ -6782,6 +6782,10 @@ get_actual_variable_endpoint(Relation heapRel, * a huge amount of time here, so we give up once we've read too many heap * pages. When we fail for that reason, the caller will end up using * whatever extremal value is recorded in pg_statistic. + * + * XXX We're not using ios_prefetch_block here. That creates a window + * where the scan's read stream can get out of sync. At a minimum we'll + * need to close this window by explicitly disabling heap I/O prefetching. */ InitNonVacuumableSnapshot(SnapshotNonVacuumable, GlobalVisTestFor(heapRel)); diff --git a/src/backend/utils/misc/guc_parameters.dat b/src/backend/utils/misc/guc_parameters.dat index 1128167c0251..79487545e591 100644 --- a/src/backend/utils/misc/guc_parameters.dat +++ b/src/backend/utils/misc/guc_parameters.dat @@ -875,6 +875,13 @@ boot_val => 'true', }, +{ name => 'enable_indexscan_prefetch', type => 'bool', context => 'PGC_USERSET', group => 'QUERY_TUNING_METHOD', + short_desc => 'Enables prefetching for index scans and index-only-scans.', + flags => 'GUC_EXPLAIN', + variable => 'enable_indexscan_prefetch', + boot_val => 'true', +}, + { name => 'enable_material', type => 'bool', context => 'PGC_USERSET', group => 'QUERY_TUNING_METHOD', short_desc => 'Enables the planner\'s use of materialization.', flags => 'GUC_EXPLAIN', diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample index f62b61967ef6..20a0ffaa5385 100644 --- a/src/backend/utils/misc/postgresql.conf.sample +++ b/src/backend/utils/misc/postgresql.conf.sample @@ -412,6 +412,7 @@ #enable_incremental_sort = on #enable_indexscan = on #enable_indexonlyscan = on +#enable_indexscan_prefetch = on #enable_material = on #enable_memoize = on #enable_mergejoin = on diff --git a/src/include/access/amapi.h b/src/include/access/amapi.h index 63dd41c1f21b..3a651744ef9a 100644 --- a/src/include/access/amapi.h +++ b/src/include/access/amapi.h @@ -198,6 +198,15 @@ typedef void (*amrescan_function) (IndexScanDesc scan, typedef bool (*amgettuple_function) (IndexScanDesc scan, ScanDirection direction); +/* next batch of valid tuples */ +typedef IndexScanBatch(*amgetbatch_function) (IndexScanDesc scan, + IndexScanBatch batch, + ScanDirection direction); + +/* release batch of valid tuples */ +typedef void (*amfreebatch_function) (IndexScanDesc scan, + IndexScanBatch batch); + /* fetch all valid tuples */ typedef int64 (*amgetbitmap_function) (IndexScanDesc scan, TIDBitmap *tbm); @@ -205,11 +214,9 @@ typedef int64 (*amgetbitmap_function) (IndexScanDesc scan, /* end index scan */ typedef void (*amendscan_function) (IndexScanDesc scan); -/* mark current scan position */ -typedef void (*ammarkpos_function) (IndexScanDesc scan); - -/* restore marked scan position */ -typedef void (*amrestrpos_function) (IndexScanDesc scan); +/* invalidate index AM state that independently tracks scan's position */ +typedef void (*amposreset_function) (IndexScanDesc scan, + IndexScanBatch batch); /* * Callback function signatures - for parallel index scans. @@ -309,10 +316,11 @@ typedef struct IndexAmRoutine ambeginscan_function ambeginscan; amrescan_function amrescan; amgettuple_function amgettuple; /* can be NULL */ + amgetbatch_function amgetbatch; /* can be NULL */ + amfreebatch_function amfreebatch; /* can be NULL */ amgetbitmap_function amgetbitmap; /* can be NULL */ amendscan_function amendscan; - ammarkpos_function ammarkpos; /* can be NULL */ - amrestrpos_function amrestrpos; /* can be NULL */ + amposreset_function amposreset; /* can be NULL */ /* interface functions to support parallel index scans */ amestimateparallelscan_function amestimateparallelscan; /* can be NULL */ diff --git a/src/include/access/genam.h b/src/include/access/genam.h index 9200a22bd9f9..767503bb68c7 100644 --- a/src/include/access/genam.h +++ b/src/include/access/genam.h @@ -15,6 +15,7 @@ #define GENAM_H #include "access/htup.h" +#include "access/itup.h" #include "access/sdir.h" #include "access/skey.h" #include "nodes/tidbitmap.h" @@ -115,6 +116,7 @@ typedef bool (*IndexBulkDeleteCallback) (ItemPointer itemptr, void *state); /* struct definitions appear in relscan.h */ typedef struct IndexScanDescData *IndexScanDesc; +typedef struct IndexScanBatchData *IndexScanBatch; typedef struct SysScanDescData *SysScanDesc; typedef struct ParallelIndexScanDescData *ParallelIndexScanDesc; @@ -228,6 +230,8 @@ extern void index_store_float8_orderby_distances(IndexScanDesc scan, bool recheckOrderBy); extern bytea *index_opclass_options(Relation indrel, AttrNumber attnum, Datum attoptions, bool validate); +extern IndexScanBatch index_batch_alloc(int maxitems, bool want_itup); +extern void index_batch_unlock(Relation rel, bool dropPin, IndexScanBatch batch); /* diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h index 909db73b7bbb..744ad2fac145 100644 --- a/src/include/access/heapam.h +++ b/src/include/access/heapam.h @@ -117,6 +117,7 @@ typedef struct IndexFetchHeapData IndexFetchTableData xs_base; /* AM independent part of the descriptor */ Buffer xs_cbuf; /* current heap buffer in scan, if any */ + BlockNumber xs_blk; /* xs_cbuf's block number, if any */ /* NB: if xs_cbuf is not InvalidBuffer, we hold a pin on that buffer */ } IndexFetchHeapData; diff --git a/src/include/access/nbtree.h b/src/include/access/nbtree.h index 16be5c7a9c15..119705f64b81 100644 --- a/src/include/access/nbtree.h +++ b/src/include/access/nbtree.h @@ -939,10 +939,10 @@ typedef BTVacuumPostingData *BTVacuumPosting; * processing. This approach minimizes lock/unlock traffic. We must always * drop the lock to make it okay for caller to process the returned items. * Whether or not we can also release the pin during this window will vary. - * We drop the pin (when so->dropPin) to avoid blocking progress by VACUUM - * (see nbtree/README section about making concurrent TID recycling safe). - * We'll always release both the lock and the pin on the current page before - * moving on to its sibling page. + * We drop the pin (when dropPin is set in batch state) to avoid blocking + * progress by VACUUM (see nbtree/README section about making concurrent TID + * recycling safe). We'll always release both the lock and the pin on the + * current page before moving on to its sibling page. * * If we are doing an index-only scan, we save the entire IndexTuple for each * matched item, otherwise only its heap TID and offset. The IndexTuples go @@ -961,74 +961,25 @@ typedef struct BTScanPosItem /* what we remember about each match */ typedef struct BTScanPosData { - Buffer buf; /* currPage buf (invalid means unpinned) */ - /* page details as of the saved position's call to _bt_readpage */ BlockNumber currPage; /* page referenced by items array */ BlockNumber prevPage; /* currPage's left link */ BlockNumber nextPage; /* currPage's right link */ - XLogRecPtr lsn; /* currPage's LSN (when so->dropPin) */ /* scan direction for the saved position's call to _bt_readpage */ ScanDirection dir; - /* - * If we are doing an index-only scan, nextTupleOffset is the first free - * location in the associated tuple storage workspace. - */ - int nextTupleOffset; - /* * moreLeft and moreRight track whether we think there may be matching * index entries to the left and right of the current page, respectively. */ bool moreLeft; bool moreRight; - - /* - * The items array is always ordered in index order (ie, increasing - * indexoffset). When scanning backwards it is convenient to fill the - * array back-to-front, so we start at the last slot and fill downwards. - * Hence we need both a first-valid-entry and a last-valid-entry counter. - * itemIndex is a cursor showing which entry was last returned to caller. - */ - int firstItem; /* first valid index in items[] */ - int lastItem; /* last valid index in items[] */ - int itemIndex; /* current index in items[] */ - - BTScanPosItem items[MaxTIDsPerBTreePage]; /* MUST BE LAST */ } BTScanPosData; typedef BTScanPosData *BTScanPos; -#define BTScanPosIsPinned(scanpos) \ -( \ - AssertMacro(BlockNumberIsValid((scanpos).currPage) || \ - !BufferIsValid((scanpos).buf)), \ - BufferIsValid((scanpos).buf) \ -) -#define BTScanPosUnpin(scanpos) \ - do { \ - ReleaseBuffer((scanpos).buf); \ - (scanpos).buf = InvalidBuffer; \ - } while (0) -#define BTScanPosUnpinIfPinned(scanpos) \ - do { \ - if (BTScanPosIsPinned(scanpos)) \ - BTScanPosUnpin(scanpos); \ - } while (0) - -#define BTScanPosIsValid(scanpos) \ -( \ - AssertMacro(BlockNumberIsValid((scanpos).currPage) || \ - !BufferIsValid((scanpos).buf)), \ - BlockNumberIsValid((scanpos).currPage) \ -) -#define BTScanPosInvalidate(scanpos) \ - do { \ - (scanpos).buf = InvalidBuffer; \ - (scanpos).currPage = InvalidBlockNumber; \ - } while (0) +#define BTScanPosIsValid(scanpos) BlockNumberIsValid((scanpos).currPage) /* We need one of these for each equality-type SK_SEARCHARRAY scan key */ typedef struct BTArrayKeyInfo @@ -1066,32 +1017,7 @@ typedef struct BTScanOpaqueData BTArrayKeyInfo *arrayKeys; /* info about each equality-type array key */ FmgrInfo *orderProcs; /* ORDER procs for required equality keys */ MemoryContext arrayContext; /* scan-lifespan context for array data */ - - /* info about killed items if any (killedItems is NULL if never used) */ - int *killedItems; /* currPos.items indexes of killed items */ - int numKilled; /* number of currently stored items */ - bool dropPin; /* drop leaf pin before btgettuple returns? */ - - /* - * If we are doing an index-only scan, these are the tuple storage - * workspaces for the currPos and markPos respectively. Each is of size - * BLCKSZ, so it can hold as much as a full page's worth of tuples. - */ - char *currTuples; /* tuple storage for currPos */ - char *markTuples; /* tuple storage for markPos */ - - /* - * If the marked position is on the same page as current position, we - * don't use markPos, but just keep the marked itemIndex in markItemIndex - * (all the rest of currPos is valid for the mark position). Hence, to - * determine if there is a mark, first look at markItemIndex, then at - * markPos. - */ - int markItemIndex; /* itemIndex, or -1 if not valid */ - - /* keep these last in struct for efficiency */ - BTScanPosData currPos; /* current position data */ - BTScanPosData markPos; /* marked position, if any */ + BTScanPos pos; } BTScanOpaqueData; typedef BTScanOpaqueData *BTScanOpaque; @@ -1191,14 +1117,15 @@ extern bool btinsert(Relation rel, Datum *values, bool *isnull, extern IndexScanDesc btbeginscan(Relation rel, int nkeys, int norderbys); extern Size btestimateparallelscan(Relation rel, int nkeys, int norderbys); extern void btinitparallelscan(void *target); -extern bool btgettuple(IndexScanDesc scan, ScanDirection dir); +extern IndexScanBatch btgetbatch(IndexScanDesc scan, IndexScanBatch batch, + ScanDirection dir); extern int64 btgetbitmap(IndexScanDesc scan, TIDBitmap *tbm); extern void btrescan(IndexScanDesc scan, ScanKey scankey, int nscankeys, ScanKey orderbys, int norderbys); +extern void btfreebatch(IndexScanDesc scan, IndexScanBatch batch); extern void btparallelrescan(IndexScanDesc scan); extern void btendscan(IndexScanDesc scan); -extern void btmarkpos(IndexScanDesc scan); -extern void btrestrpos(IndexScanDesc scan); +extern void btposreset(IndexScanDesc scan, IndexScanBatch markbatch); extern IndexBulkDeleteResult *btbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats, IndexBulkDeleteCallback callback, @@ -1306,8 +1233,9 @@ extern BTStack _bt_search(Relation rel, Relation heaprel, BTScanInsert key, Buffer *bufP, int access); extern OffsetNumber _bt_binsrch_insert(Relation rel, BTInsertState insertstate); extern int32 _bt_compare(Relation rel, BTScanInsert key, Page page, OffsetNumber offnum); -extern bool _bt_first(IndexScanDesc scan, ScanDirection dir); -extern bool _bt_next(IndexScanDesc scan, ScanDirection dir); +extern IndexScanBatch _bt_first(IndexScanDesc scan, ScanDirection dir); +extern IndexScanBatch _bt_next(IndexScanDesc scan, ScanDirection dir, + IndexScanBatch priorbatch); extern Buffer _bt_get_endpoint(Relation rel, uint32 level, bool rightmost); /* @@ -1327,7 +1255,7 @@ extern bool _bt_checkkeys(IndexScanDesc scan, BTReadPageState *pstate, bool arra extern bool _bt_scanbehind_checkkeys(IndexScanDesc scan, ScanDirection dir, IndexTuple finaltup); extern void _bt_set_startikey(IndexScanDesc scan, BTReadPageState *pstate); -extern void _bt_killitems(IndexScanDesc scan); +extern void _bt_killitems(IndexScanDesc scan, IndexScanBatch batch); extern BTCycleId _bt_vacuum_cycleid(Relation rel); extern BTCycleId _bt_start_vacuum(Relation rel); extern void _bt_end_vacuum(Relation rel); diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h index b5e0fb386c0a..6f87c6b313b1 100644 --- a/src/include/access/relscan.h +++ b/src/include/access/relscan.h @@ -16,9 +16,11 @@ #include "access/htup_details.h" #include "access/itup.h" +#include "access/sdir.h" #include "nodes/tidbitmap.h" #include "port/atomics.h" #include "storage/buf.h" +#include "storage/read_stream.h" #include "storage/relfilelocator.h" #include "storage/spin.h" #include "utils/relcache.h" @@ -121,10 +123,156 @@ typedef struct ParallelBlockTableScanWorkerData *ParallelBlockTableScanWorker; typedef struct IndexFetchTableData { Relation rel; + ReadStream *rs; } IndexFetchTableData; struct IndexScanInstrumentation; +/* Forward declaration, the prefetch callback needs IndexScanDescData. */ +typedef struct IndexScanBatchData IndexScanBatchData; + +typedef struct IndexScanBatchPosItem /* what we remember about each match */ +{ + ItemPointerData heapTid; /* TID of referenced heap item */ + OffsetNumber indexOffset; /* index item's location within page */ + LocationIndex tupleOffset; /* IndexTuple's offset in workspace, if any */ +} IndexScanBatchPosItem; + +/* + * Data about one batch of items returned by the index AM + */ +typedef struct IndexScanBatchData +{ + Buffer buf; /* currPage buf (invalid means unpinned) */ + XLogRecPtr lsn; /* currPage's LSN (when dropPin) */ + + /* + * AM-specific state representing the current position of the scan within + * the index + */ + void *pos; + + /* + * The items array is always ordered in index order (ie, increasing + * indexoffset). When scanning backwards it is convenient to fill the + * array back-to-front, so we start at the last slot and fill downwards. + * Hence we need both a first-valid-entry and a last-valid-entry counter. + */ + int firstItem; /* first valid index in items[] */ + int lastItem; /* last valid index in items[] */ + + /* info about killed items if any (killedItems is NULL if never used) */ + int *killedItems; /* indexes of killed items */ + int numKilled; /* number of currently stored items */ + + /* + * If we are doing an index-only scan, these are the tuple storage + * workspaces for the matching tuples (tuples referenced by items[]). Each + * is of size BLCKSZ, so it can hold as much as a full page's worth of + * tuples. + * + * XXX maybe currTuples should be part of the am-specific per-batch state + * stored in "position" field? + */ + char *currTuples; /* tuple storage for items[] */ + + /* + * batch contents (TIDs, index tuples, kill bitmap, ...) + * + * XXX Shouldn't this be part of the "IndexScanBatchPosItem" struct? To + * keep everything in one place? Or why should we have separate arrays? + * One advantage is that we don't need to allocate memory for arrays that + * we don't need ... e.g. if we don't need heap tuples, we don't allocate + * that. We couldn't do that with everything in one struct. + */ + char *itemsvisibility; /* Index-only scan visibility cache */ + + IndexScanBatchPosItem items[FLEXIBLE_ARRAY_MEMBER]; +} IndexScanBatchData; + +/* + * Position in the queue of batches - index of a batch, index of item in a batch. + */ +typedef struct IndexScanBatchPos +{ + int batch; + int index; +} IndexScanBatchPos; + +typedef struct IndexScanDescData IndexScanDescData; +typedef bool (*IndexPrefetchCallback) (IndexScanDescData * scan, + void *arg, + IndexScanBatchPos *pos); + +/* + * State used by amgetbatch index AMs, which manage per-page batches of items + * with matching index tuples using a circular buffer + */ +typedef struct IndexScanBatchState +{ + /* Index AM drops leaf pin before amgetbatch returns? */ + bool dropPin; + + /* + * Did we read the final batch in this scan direction? The batches may be + * loaded from multiple places, and we need to remember when we fail to + * load the next batch in a given scan (which means "no more batches"). + * amgetbatch may restart the scan on the get call, so we need to remember + * it's over. + */ + bool finished; + bool reset; + + /* + * Did we disable prefetching/use of a read stream because it didn't pay + * for itself? + */ + bool prefetchingLockedIn; + bool disabled; + + /* + * During prefetching, currentPrefetchBlock is the table AM block number + * that was returned by our read stream callback most recently. Used to + * suppress duplicate successive read stream block requests. + * + * Prefetching can still perform non-successive requests for the same + * block number (in general we're prefetching in exactly the same order + * that the scan will return table AM TIDs in). We need to avoid + * duplicate successive requests because table AMs expect to be able to + * hang on to buffer pins across table_index_fetch_tuple calls. + */ + BlockNumber currentPrefetchBlock; + + /* + * Current scan direction, for the currently loaded batches. This is used + * to load data in the read stream API callback, etc. + */ + ScanDirection direction; + + /* positions in the queue of batches (batch + item) */ + IndexScanBatchPos readPos; /* read position */ + IndexScanBatchPos streamPos; /* prefetch position (for read stream API) */ + IndexScanBatchPos markPos; /* mark/restore position */ + + IndexScanBatchData *markBatch; + + /* + * Array of batches returned by the AM. The array has a capacity (but can + * be resized if needed). The headBatch is an index of the batch we're + * currently reading from (this needs to be translated by modulo + * maxBatches into index in the batches array). + */ + int maxBatches; /* size of the batches array */ + int headBatch; /* head batch slot */ + int nextBatch; /* next empty batch slot */ + + IndexScanBatchData **batches; + + /* callback to skip prefetching in IOS etc. */ + IndexPrefetchCallback prefetch; + void *prefetchArg; +} IndexScanBatchState; + /* * We use the same IndexScanDescData structure for both amgettuple-based * and amgetbitmap-based index scans. Some fields are only relevant in @@ -138,6 +286,8 @@ typedef struct IndexScanDescData struct SnapshotData *xs_snapshot; /* snapshot to see */ int numberOfKeys; /* number of index qualifier conditions */ int numberOfOrderBys; /* number of ordering operators */ + IndexScanBatchState *batchState; /* amgetbatch related state */ + struct ScanKeyData *keyData; /* array of index qualifier descriptors */ struct ScanKeyData *orderByData; /* array of ordering op descriptors */ bool xs_want_itup; /* caller requests index tuples */ diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h index 30d889b54c53..6879fe99b4bf 100644 --- a/src/include/nodes/pathnodes.h +++ b/src/include/nodes/pathnodes.h @@ -1343,7 +1343,7 @@ typedef struct IndexOptInfo /* does AM have amgetbitmap interface? */ bool amhasgetbitmap; bool amcanparallel; - /* does AM have ammarkpos interface? */ + /* is AM prepared for us to restore a mark? */ bool amcanmarkpos; /* AM's cost estimator */ /* Rather than include amapi.h here, we declare amcostestimate like this */ diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h index b523bcda8f3d..00f4c3d00118 100644 --- a/src/include/optimizer/cost.h +++ b/src/include/optimizer/cost.h @@ -51,6 +51,7 @@ extern PGDLLIMPORT Cost disable_cost; extern PGDLLIMPORT int max_parallel_workers_per_gather; extern PGDLLIMPORT bool enable_seqscan; extern PGDLLIMPORT bool enable_indexscan; +extern PGDLLIMPORT bool enable_indexscan_prefetch; extern PGDLLIMPORT bool enable_indexonlyscan; extern PGDLLIMPORT bool enable_bitmapscan; extern PGDLLIMPORT bool enable_tidscan; diff --git a/src/test/modules/dummy_index_am/dummy_index_am.c b/src/test/modules/dummy_index_am/dummy_index_am.c index 94ef639b6fcd..be1b0f55ca7e 100644 --- a/src/test/modules/dummy_index_am/dummy_index_am.c +++ b/src/test/modules/dummy_index_am/dummy_index_am.c @@ -319,8 +319,7 @@ dihandler(PG_FUNCTION_ARGS) amroutine->amgettuple = NULL; amroutine->amgetbitmap = NULL; amroutine->amendscan = diendscan; - amroutine->ammarkpos = NULL; - amroutine->amrestrpos = NULL; + amroutine->amposreset = NULL; amroutine->amestimateparallelscan = NULL; amroutine->aminitparallelscan = NULL; amroutine->amparallelrescan = NULL; diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out index 3b37fafa65b9..9702e3103955 100644 --- a/src/test/regress/expected/sysviews.out +++ b/src/test/regress/expected/sysviews.out @@ -159,6 +159,7 @@ select name, setting from pg_settings where name like 'enable%'; enable_incremental_sort | on enable_indexonlyscan | on enable_indexscan | on + enable_indexscan_prefetch | on enable_material | on enable_memoize | on enable_mergejoin | on @@ -173,7 +174,7 @@ select name, setting from pg_settings where name like 'enable%'; enable_seqscan | on enable_sort | on enable_tidscan | on -(25 rows) +(26 rows) -- There are always wait event descriptions for various types. InjectionPoint -- may be present or absent, depending on history since last postmaster start. diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list index 432509277c98..d535df692fc8 100644 --- a/src/tools/pgindent/typedefs.list +++ b/src/tools/pgindent/typedefs.list @@ -193,6 +193,8 @@ BOOL BOOLEAN BOX BTArrayKeyInfo +BTBatchInfo +BTBatchScanPosData BTBuildState BTCallbackState BTCycleId @@ -1269,6 +1271,10 @@ IndexOrderByDistance IndexPath IndexRuntimeKeyInfo IndexScan +IndexScanBatchData +IndexScanBatchPos +IndexScanBatchPosItem +IndexScanBatches IndexScanDesc IndexScanInstrumentation IndexScanState @@ -3432,10 +3438,10 @@ amestimateparallelscan_function amgetbitmap_function amgettreeheight_function amgettuple_function +amgetbatch_function aminitparallelscan_function aminsert_function aminsertcleanup_function -ammarkpos_function amoptions_function amparallelrescan_function amproperty_function From 0d15bff9ed7885f5c07c6c07aea3ded20d1b9d71 Mon Sep 17 00:00:00 2001 From: Peter Geoghegan Date: Wed, 10 Sep 2025 16:54:50 -0400 Subject: [PATCH 3/3] Reduce malloc/free traffic by caching batches Instead of immediately freeing a batch, stash it in a cache (a small fixed-size array), for reuse by the same scan. There's room for improvement: - Keeping some of the batch pieces (killItems, itemsvisibility, ...) instead of freeing them in index_batch_release. - Allocating only space we need (both index_batch_alloc calls use MaxTIDsPerBTreePage, and thus malloc - because of ALLOC_CHUNK_LIMIT). --- src/backend/access/index/indexam.c | 202 ++++++++++++++++++++++++-- src/backend/access/nbtree/nbtree.c | 16 +- src/backend/access/nbtree/nbtsearch.c | 4 +- src/include/access/genam.h | 4 +- src/include/access/relscan.h | 8 + 5 files changed, 210 insertions(+), 24 deletions(-) diff --git a/src/backend/access/index/indexam.c b/src/backend/access/index/indexam.c index fd3ffa222bec..c32f82a3b485 100644 --- a/src/backend/access/index/indexam.c +++ b/src/backend/access/index/indexam.c @@ -2196,6 +2196,10 @@ index_batch_init(IndexScanDesc scan) scan->batchState->headBatch = 0; /* initial head batch */ scan->batchState->nextBatch = 0; /* initial batch starts empty */ + /* XXX init the cache of batches, capacity 16 is arbitrary */ + scan->batchState->cache.maxbatches = 16; + scan->batchState->cache.batches = NULL; + scan->batchState->batches = palloc(sizeof(IndexScanBatchData *) * scan->batchState->maxBatches); @@ -2328,34 +2332,123 @@ static void index_batch_end(IndexScanDesc scan) { index_batch_reset(scan, true); + + /* bail out without batching */ + if (!scan->batchState) + return; + + /* we can simply free batches thanks to the earlier reset */ + if (scan->batchState->batches) + pfree(scan->batchState->batches); + + /* also walk the cache of batches, if any */ + if (scan->batchState->cache.batches) + { + for (int i = 0; i < scan->batchState->cache.maxbatches; i++) + { + if (scan->batchState->cache.batches[i] == NULL) + continue; + + pfree(scan->batchState->cache.batches[i]); + } + + pfree(scan->batchState->cache.batches); + } + + pfree(scan->batchState); } /* + * index_batch_alloc + * Allocate a batch that can fit maxitems index tuples. + * + * Returns a IndexScanBatch struct with capacity sufficient for maxitems + * index tuples. It's either newly allocated or loaded from a small cache + * maintained for individual scans. + * + * maxitems determines the minimum size of the batch (it may be larger) + * want_itup determines whether the bach allocates space for currTuples + * * XXX Both index_batch_alloc() calls in btree use MaxTIDsPerBTreePage, * which seems unfortunate - it increases the allocation sizes, even if - * the index would be fine with smaller arrays. This means all batches exceed - * ALLOC_CHUNK_LIMIT, forcing a separate malloc (expensive). + * the index would be fine with smaller arrays. This means all batches + * exceed ALLOC_CHUNK_LIMIT, forcing a separate malloc (expensive). The + * cache helps for longer queries, not for queries that only create a + * single batch, etc. */ IndexScanBatch -index_batch_alloc(int maxitems, bool want_itup) +index_batch_alloc(IndexScanDesc scan, int maxitems, bool want_itup) { - IndexScanBatch batch = palloc(offsetof(IndexScanBatchData, items) + - sizeof(IndexScanBatchPosItem) * maxitems); + IndexScanBatch batch = NULL; + + /* + * Try to find a sufficiently large batch in the cache. + * + * Use the first batch that can fit the requested number of items. We + * could be smarter and look for the smallest of such batches. But that + * probably won't help very much. We expect batches to be mostly uniform, + * with about the same size. And index_batch_release() prefers larger + * batches, so we should end up with mostly larger batches in the cache. + * + * XXX We can get here with batchState==NULL for bitmapscans. Could that + * mean bitmapscans have issues with malloc/free on batches too? But the + * cache can't help with that, when it's in batchState (because bitmap + * scans don't have that). + */ + if ((scan->batchState != NULL) && + (scan->batchState->cache.batches != NULL)) + { + /* + * try to find a batch in the cache, with maxitems high enough + * + * XXX Maybe should look for a batch with lowest maxitems? That should + * increase probability of cache hits in the future? + */ + for (int i = 0; i < scan->batchState->cache.maxbatches; i++) + { + if ((scan->batchState->cache.batches[i] != NULL) && + (scan->batchState->cache.batches[i]->maxitems >= maxitems)) + { + batch = scan->batchState->cache.batches[i]; + scan->batchState->cache.batches[i] = NULL; + break; + } + } + } + + /* found a batch in the cache? */ + if (batch) + { + /* for IOS, we expect to already have the currTuples */ + Assert(!(want_itup && (batch->currTuples == NULL))); + + /* XXX maybe we could keep these allocations too */ + Assert(batch->pos == NULL); + Assert(batch->itemsvisibility == NULL); + } + else + { + batch = palloc(offsetof(IndexScanBatchData, items) + + sizeof(IndexScanBatchPosItem) * maxitems); + + batch->maxitems = maxitems; + + /* + * If we are doing an index-only scan, we need a tuple storage + * workspace. We allocate BLCKSZ for this, which should always give + * the index AM enough space to fit a full page's worth of tuples. + */ + batch->currTuples = NULL; + if (want_itup) + batch->currTuples = palloc(BLCKSZ); + } + /* shared initialization */ batch->firstItem = -1; batch->lastItem = -1; batch->killedItems = NULL; batch->numKilled = 0; - /* - * If we are doing an index-only scan, we need a tuple storage workspace. - * We allocate BLCKSZ for this, which should always give the index AM - * enough space to fit a full page's worth of tuples. - */ - batch->currTuples = NULL; - if (want_itup) - batch->currTuples = palloc(BLCKSZ); - batch->buf = InvalidBuffer; batch->pos = NULL; batch->itemsvisibility = NULL; /* per-batch IOS visibility */ @@ -2396,3 +2489,84 @@ index_batch_unlock(Relation rel, bool dropPin, IndexScanBatch batch) ReleaseBuffer(batch->buf); batch->buf = InvalidBuffer; /* defensive */ } + +/* + * index_batch_release + * Either stash the batch info a small cache for reuse, or free it. + */ +void +index_batch_release(IndexScanDesc scan, IndexScanBatch batch) +{ + /* custom fields should have been cleaned by amfreebatch */ + Assert(batch->pos == NULL); + Assert(batch->buf == InvalidBuffer); + + /* + * free killedItems / itemvisibility + * + * XXX We could keep/reuse those too, I guess. + */ + + if (batch->killedItems != NULL) + { + pfree(batch->killedItems); + batch->killedItems = NULL; + } + + if (batch->itemsvisibility != NULL) + { + pfree(batch->itemsvisibility); + batch->itemsvisibility = NULL; + } + + /* + * Try adding the batch to the small cache - find a slot that's either empty + * or used by a smaller batch (with smallest maxitems value), and replace + * that batch). + * + * XXX There may be ways to improve this. We could track the number of empty + * slots, and minimum maxitems value, which would allow skipping pointless + * searches (in cases when should just discard the batch). + */ + if (scan->batchState != NULL) + { + /* lowest maxitems we found in the cache (to replace with this batch) */ + int maxitems = batch->maxitems; + int slot = scan->batchState->cache.maxbatches; + + /* first time through, initialize the cache */ + if (scan->batchState->cache.batches == NULL) + scan->batchState->cache.batches + = palloc0_array(IndexScanBatch, + scan->batchState->cache.maxbatches); + + /* find am empty or sufficiently large batch */ + for (int i = 0; i < scan->batchState->cache.maxbatches; i++) + { + /* found empty slot, we're done */ + if (scan->batchState->cache.batches[i] == NULL) + { + scan->batchState->cache.batches[i] = batch; + return; + } + + /* found a smaller slot, remember it */ + if (scan->batchState->cache.batches[i]->maxitems < maxitems) + { + maxitems = scan->batchState->cache.batches[i]->maxitems; + slot = i; + } + } + + /* found a slot for this batch? */ + if (maxitems < batch->maxitems) + { + pfree(scan->batchState->cache.batches[slot]); + scan->batchState->cache.batches[slot] = batch; + return; + } + } + + /* either no cache or no slot for this batch */ + pfree(batch); +} diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c index fba562df8a09..af947b6dcaa9 100644 --- a/src/backend/access/nbtree/nbtree.c +++ b/src/backend/access/nbtree/nbtree.c @@ -382,21 +382,23 @@ btfreebatch(IndexScanDesc scan, IndexScanBatch batch) if (batch->numKilled > 0) _bt_killitems(scan, batch); - if (batch->itemsvisibility) - pfree(batch->itemsvisibility); - - if (batch->currTuples) - pfree(batch->currTuples); - + /* free AM-specific fields of the batch */ if (batch->pos) { if (!scan->batchState || !scan->batchState->dropPin) + { ReleaseBuffer(batch->buf); + batch->buf = InvalidBuffer; + } pfree(batch->pos); + batch->pos = NULL; } - pfree(batch); + /* other fields (itemsvisibility, killItems, currTuples) freed elsewhere */ + + /* free the batch (or cache it for reuse) */ + index_batch_release(scan, batch); } /* diff --git a/src/backend/access/nbtree/nbtsearch.c b/src/backend/access/nbtree/nbtsearch.c index 54a767a4fe41..b019c19f806d 100644 --- a/src/backend/access/nbtree/nbtsearch.c +++ b/src/backend/access/nbtree/nbtsearch.c @@ -1200,7 +1200,7 @@ _bt_first(IndexScanDesc scan, ScanDirection dir) } /* Allocate space for first batch */ - firstbatch = index_batch_alloc(MaxTIDsPerBTreePage, scan->xs_want_itup); + firstbatch = index_batch_alloc(scan, MaxTIDsPerBTreePage, scan->xs_want_itup); firstbatch->pos = palloc(sizeof(BTScanPosData)); /* @@ -2237,7 +2237,7 @@ _bt_readnextpage(IndexScanDesc scan, BlockNumber blkno, BTScanPos newpos; /* Allocate space for next batch */ - newbatch = index_batch_alloc(MaxTIDsPerBTreePage, scan->xs_want_itup); + newbatch = index_batch_alloc(scan, MaxTIDsPerBTreePage, scan->xs_want_itup); newbatch->pos = palloc(sizeof(BTScanPosData)); newpos = newbatch->pos; diff --git a/src/include/access/genam.h b/src/include/access/genam.h index 767503bb68c7..1a92a8195efc 100644 --- a/src/include/access/genam.h +++ b/src/include/access/genam.h @@ -230,7 +230,9 @@ extern void index_store_float8_orderby_distances(IndexScanDesc scan, bool recheckOrderBy); extern bytea *index_opclass_options(Relation indrel, AttrNumber attnum, Datum attoptions, bool validate); -extern IndexScanBatch index_batch_alloc(int maxitems, bool want_itup); +extern IndexScanBatch index_batch_alloc(IndexScanDesc scan, + int maxitems, bool want_itup); +extern void index_batch_release(IndexScanDesc scan, IndexScanBatch batch); extern void index_batch_unlock(Relation rel, bool dropPin, IndexScanBatch batch); diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h index 6f87c6b313b1..3a0f4dce6eb2 100644 --- a/src/include/access/relscan.h +++ b/src/include/access/relscan.h @@ -187,6 +187,8 @@ typedef struct IndexScanBatchData */ char *itemsvisibility; /* Index-only scan visibility cache */ + /* capacity of the batch (size of the items array) */ + int maxitems; IndexScanBatchPosItem items[FLEXIBLE_ARRAY_MEMBER]; } IndexScanBatchData; @@ -266,6 +268,12 @@ typedef struct IndexScanBatchState int headBatch; /* head batch slot */ int nextBatch; /* next empty batch slot */ + /* small cache of unused batches, to reduce malloc/free traffic */ + struct { + int maxbatches; + IndexScanBatchData **batches; + } cache; + IndexScanBatchData **batches; /* callback to skip prefetching in IOS etc. */