From 9804077aca1920d08e1007a32387c72f0fdea7ff Mon Sep 17 00:00:00 2001 From: Mikhail Nikalayeu Date: Tue, 31 Dec 2024 15:03:10 +0100 Subject: [PATCH v26 5/8] Use auxiliary indexes for concurrent index operations Replace the second table full scan in concurrent index builds with an auxiliary index approach: - create a STIR auxiliary index with the same predicate (if exists) as in main index - use it to track tuples inserted during the first phase - merge auxiliary index with main index during validation to catch up new index with any tuples missed during the first phase - automatically drop auxiliary when main index is ready To merge main and auxiliary indexes: - index_bulk_delete called for both, TIDs put into tuplesort - both tuplesort are being sorted - both tuplesort scanned with two pointers looking for the TIDs present in auxiliary index, but absent in main one - all such TIDs are put into tuplestore - all TIDs in tuplestore are fetched using the stream, tuplestore used in heapam_index_validate_scan_read_stream_next to provide the next page to prefetch - if fetched tuple is alive - it is inserted into the main index This eliminates the need for a second full table scan during validation, improving performance especially for large tables. Affects both CREATE INDEX CONCURRENTLY and REINDEX INDEX CONCURRENTLY operations. --- doc/src/sgml/monitoring.sgml | 26 +- doc/src/sgml/ref/create_index.sgml | 34 +- doc/src/sgml/ref/reindex.sgml | 41 +- src/backend/access/heap/README.HOT | 13 +- src/backend/access/heap/heapam_handler.c | 544 ++++++++++++++------- src/backend/catalog/index.c | 314 ++++++++++-- src/backend/catalog/system_views.sql | 17 +- src/backend/commands/indexcmds.c | 344 +++++++++++-- src/backend/nodes/makefuncs.c | 4 +- src/include/access/tableam.h | 12 +- src/include/catalog/index.h | 9 +- src/include/commands/progress.h | 13 +- src/include/nodes/makefuncs.h | 3 +- src/test/regress/expected/create_index.out | 42 ++ src/test/regress/expected/indexing.out | 3 +- src/test/regress/expected/rules.out | 17 +- src/test/regress/sql/create_index.sql | 21 + 17 files changed, 1123 insertions(+), 334 deletions(-) diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml index 2741c138593..868b025e2ed 100644 --- a/doc/src/sgml/monitoring.sgml +++ b/doc/src/sgml/monitoring.sgml @@ -6407,6 +6407,18 @@ FROM pg_stat_get_backend_idset() AS backendid; information for this phase. + + waiting for writers to use auxiliary index + + CREATE INDEX CONCURRENTLY or REINDEX CONCURRENTLY is waiting for transactions + with write locks that can potentially see the table to finish, to ensure use of auxiliary index for new tuples in + future transactions. + This phase is skipped when not in concurrent mode. + Columns lockers_total, lockers_done + and current_locker_pid contain the progress + information for this phase. + + building index @@ -6447,13 +6459,12 @@ FROM pg_stat_get_backend_idset() AS backendid; - index validation: scanning table + index validation: merging indexes - CREATE INDEX CONCURRENTLY is scanning the table - to validate the index tuples collected in the previous two phases. + CREATE INDEX CONCURRENTLY merging content of auxiliary index with the target index. This phase is skipped when not in concurrent mode. - Columns blocks_total (set to the total size of the table) - and blocks_done contain the progress information for this phase. + Columns tuples_total (set to the number of tuples to be merged) + and tuples_done contain the progress information for this phase. @@ -6470,8 +6481,9 @@ FROM pg_stat_get_backend_idset() AS backendid; waiting for readers before marking dead - REINDEX CONCURRENTLY is waiting for transactions - with read locks on the table to finish, before marking the old index dead. + CREATE INDEX CONCURRENTLY is waiting for transactions + with read locks on the table to finish, before marking the auxiliary index as dead. + REINDEX CONCURRENTLY is also waiting before marking the old index as dead. This phase is skipped when not in concurrent mode. Columns lockers_total, lockers_done and current_locker_pid contain the progress diff --git a/doc/src/sgml/ref/create_index.sgml b/doc/src/sgml/ref/create_index.sgml index bb7505d171b..ba387f28977 100644 --- a/doc/src/sgml/ref/create_index.sgml +++ b/doc/src/sgml/ref/create_index.sgml @@ -620,10 +620,10 @@ CREATE [ UNIQUE ] INDEX [ CONCURRENTLY ] [ [ IF NOT EXISTS ] CONCURRENTLY option of CREATE INDEX. When this option is used, - PostgreSQL must perform two scans of the table, and in - addition it must wait for all existing transactions that could potentially - modify or use the index to terminate. Thus - this method requires more total work than a standard index build and takes + PostgreSQL must perform table scan followed by + validation phase, and in addition it must wait for all existing transactions + that could potentially modify or use the index to terminate. Thus + this method requires more total work than a standard index build and may take significantly longer to complete. However, since it allows normal operations to continue while the index is built, this method is useful for adding new indexes in a production environment. Of course, the extra CPU @@ -631,14 +631,14 @@ CREATE [ UNIQUE ] INDEX [ CONCURRENTLY ] [ [ IF NOT EXISTS ] - In a concurrent index build, the index is actually entered as an - invalid index into - the system catalogs in one transaction, then two table scans occur in - two more transactions. Before each table scan, the index build must + In a concurrent index build, the main and auxiliary indexes is actually + entered as an invalid index into + the system catalogs in one transaction, then two phases occur in + multiple transactions. Before each phase, the index build must wait for existing transactions that have modified the table to terminate. - After the second scan, the index build must wait for any transactions + After the second phase, the index build must wait for any transactions that have a snapshot (see ) predating the second - scan to terminate, including transactions used by any phase of concurrent + phase to terminate, including transactions used by any phase of concurrent index builds on other tables, if the indexes involved are partial or have columns that are not simple column references. Then finally the index can be marked valid and ready for use, @@ -651,10 +651,11 @@ CREATE [ UNIQUE ] INDEX [ CONCURRENTLY ] [ [ IF NOT EXISTS ] If a problem arises while scanning the table, such as a deadlock or a uniqueness violation in a unique index, the CREATE INDEX - command will fail but leave behind an invalid index. This index - will be ignored for querying purposes because it might be incomplete; - however it will still consume update overhead. The psql - \d command will report such an index as INVALID: + command will fail but leave behind an invalid index and its + associated auxiliary index. These indexes + will be ignored for querying purposes because they might be incomplete; + however they will still consume update overhead. The psql + \d command will report such indexes as INVALID: postgres=# \d tab @@ -664,11 +665,12 @@ postgres=# \d tab col | integer | | | Indexes: "idx" btree (col) INVALID + "idx_ccaux" stir (col) INVALID The recommended recovery - method in such cases is to drop the index and try again to perform - CREATE INDEX CONCURRENTLY. (Another possibility is + method in such cases is to drop these indexes and try again to perform + CREATE INDEX CONCURRENTLY. (Another possibility is to rebuild the index with REINDEX INDEX CONCURRENTLY). diff --git a/doc/src/sgml/ref/reindex.sgml b/doc/src/sgml/ref/reindex.sgml index 185cd75ca30..97f551a55a6 100644 --- a/doc/src/sgml/ref/reindex.sgml +++ b/doc/src/sgml/ref/reindex.sgml @@ -368,9 +368,8 @@ REINDEX [ ( option [, ...] ) ] { DA PostgreSQL supports rebuilding indexes with minimum locking of writes. This method is invoked by specifying the CONCURRENTLY option of REINDEX. When this option - is used, PostgreSQL must perform two scans of the table - for each index that needs to be rebuilt and wait for termination of - all existing transactions that could potentially use the index. + is used, PostgreSQL must perform several steps to ensure data + consistency while allowing normal operations to continue. This method requires more total work than a standard index rebuild and takes significantly longer to complete as it needs to wait for unfinished transactions that might modify the index. However, since @@ -388,7 +387,7 @@ REINDEX [ ( option [, ...] ) ] { DA - A new transient index definition is added to the catalog + A new transient index definition and an auxiliary index are added to the catalog pg_index. This definition will be used to replace the old index. A SHARE UPDATE EXCLUSIVE lock at session level is taken on the indexes being reindexed as well as their @@ -398,7 +397,15 @@ REINDEX [ ( option [, ...] ) ] { DA - A first pass to build the index is done for each new index. Once the + The auxiliary index is marked as "ready for inserts", making + it visible to other sessions. This index efficiently tracks all new + tuples during the reindex process. + + + + + + The new main index is built by scanning the table. Once the index is built, its flag pg_index.indisready is switched to true to make it ready for inserts, making it visible to other sessions once the transaction that performed the build @@ -409,9 +416,9 @@ REINDEX [ ( option [, ...] ) ] { DA - Then a second pass is performed to add tuples that were added while the - first pass was running. This step is also done in a separate - transaction for each index. + A validation phase merges any missing entries from the auxiliary index + into the main index, ensuring all concurrent changes are captured. + This step is also done in a separate transaction for each index. @@ -428,7 +435,7 @@ REINDEX [ ( option [, ...] ) ] { DA - The old indexes have pg_index.indisready switched to + The old and auxiliary indexes have pg_index.indisready switched to false to prevent any new tuple insertions, after waiting for running queries that might reference the old index to complete. @@ -436,7 +443,7 @@ REINDEX [ ( option [, ...] ) ] { DA - The old indexes are dropped. The SHARE UPDATE + The old and auxiliary indexes are dropped. The SHARE UPDATE EXCLUSIVE session locks for the indexes and the table are released. @@ -447,11 +454,11 @@ REINDEX [ ( option [, ...] ) ] { DA If a problem arises while rebuilding the indexes, such as a uniqueness violation in a unique index, the REINDEX - command will fail but leave behind an invalid new index in addition to - the pre-existing one. This index will be ignored for querying purposes - because it might be incomplete; however it will still consume update + command will fail but leave behind an invalid new index and its auxiliary index in addition to + the pre-existing one. These indexes will be ignored for querying purposes + because they might be incomplete; however they will still consume update overhead. The psql \d command will report - such an index as INVALID: + such indexes as INVALID: postgres=# \d tab @@ -462,12 +469,14 @@ postgres=# \d tab Indexes: "idx" btree (col) "idx_ccnew" btree (col) INVALID + "idx_ccaux" stir (col) INVALID + If the index marked INVALID is suffixed - _ccnew, then it corresponds to the transient + _ccnew or _ccaux, then it corresponds to the transient or auxiliary index created during the concurrent operation, and the recommended - recovery method is to drop it using DROP INDEX, + recovery method is to drop these indexes using DROP INDEX, then attempt REINDEX CONCURRENTLY again. If the invalid index is instead suffixed _ccold, it corresponds to the original index which could not be dropped; diff --git a/src/backend/access/heap/README.HOT b/src/backend/access/heap/README.HOT index 74e407f375a..28e2a1604c4 100644 --- a/src/backend/access/heap/README.HOT +++ b/src/backend/access/heap/README.HOT @@ -375,6 +375,11 @@ constraint on which updates can be HOT. Other transactions must include such an index when determining HOT-safety of updates, even though they must ignore it for both insertion and searching purposes. +Also, special auxiliary index is created the same way. It marked as +"ready for inserts" without any actual table scan. Its purpose is collect +new tuples inserted into table while our target index is still "not ready +for inserts" + We must do this to avoid making incorrect index entries. For example, suppose we are building an index on column X and we make an index entry for a non-HOT tuple with X=1. Then some other backend, unaware that X is an @@ -394,10 +399,10 @@ entry at the root of the HOT-update chain but we use the key value from the live tuple. We mark the index open for inserts (but still not ready for reads) then -we again wait for transactions which have the table open. Then we take -a second reference snapshot and validate the index. This searches for -tuples missing from the index, and inserts any missing ones. Again, -the index entries have to have TIDs equal to HOT-chain root TIDs, but +we again wait for transactions which have the table open. Then validate +the index. This searches for tuples missing from the index in auxiliary +index, and inserts any missing ones if them visible to reference snapshot. +Again, the index entries have to have TIDs equal to HOT-chain root TIDs, but the value to be inserted is the one from the live tuple. Then we wait until every transaction that could have a snapshot older than diff --git a/src/backend/access/heap/heapam_handler.c b/src/backend/access/heap/heapam_handler.c index bcbac844bb6..c85e5332ba2 100644 --- a/src/backend/access/heap/heapam_handler.c +++ b/src/backend/access/heap/heapam_handler.c @@ -41,6 +41,7 @@ #include "storage/bufpage.h" #include "storage/lmgr.h" #include "storage/predicate.h" +#include "storage/proc.h" #include "storage/procarray.h" #include "storage/smgr.h" #include "utils/builtins.h" @@ -1743,242 +1744,405 @@ heapam_index_build_range_scan(Relation heapRelation, return reltuples; } +/* + * Calculate set difference (relative complement) of main and aux + * sets. + * + * All records which are present in auxliary tuplesort but not is + * main are added to the store. + * + * In set theory notation store = aux - main or store = aux / main. + * + * returns number of items added to store + */ +static int +heapam_index_validate_tuplesort_difference(Tuplesortstate *main, + Tuplesortstate *aux, + Tuplestorestate *store) +{ + int num = 0; + /* state variables for the merge */ + ItemPointer indexcursor = NULL, + auxindexcursor = NULL; + ItemPointerData decoded, + auxdecoded; + bool tuplesort_empty = false, + auxtuplesort_empty = false; + + /* Initialize pointers. */ + ItemPointerSetInvalid(&decoded); + ItemPointerSetInvalid(&auxdecoded); + + /* + * Main loop: we step through the auxiliary sort (auxState->tuplesort), + * which holds TIDs that must compared to those from the "main" sort + * (state->tuplesort). + */ + while (!auxtuplesort_empty) + { + Datum ts_val; + bool ts_isnull; + CHECK_FOR_INTERRUPTS(); + + /* + * Attempt to fetch the next TID from the auxiliary sort. If it's + * empty, we set auxindexcursor to NULL. + */ + auxtuplesort_empty = !tuplesort_getdatum(aux, true, + false, &ts_val, &ts_isnull, + NULL); + Assert(auxtuplesort_empty || !ts_isnull); + if (!auxtuplesort_empty) + { + itemptr_decode(&auxdecoded, DatumGetInt64(ts_val)); + auxindexcursor = &auxdecoded; + } + else + { + auxindexcursor = NULL; + } + + /* + * If the auxiliary sort is not yet empty, we now try to synchronize + * the "main" sort cursor (indexcursor) with auxindexcursor. We advance + * the main sort cursor until we've reached or passed the auxiliary TID. + */ + if (!auxtuplesort_empty) + { + /* + * Move the main sort forward while: + * (1) It's not exhausted (tuplesort_empty == false), and + * (2) Either indexcursor is NULL (first iteration) or + * indexcursor < auxindexcursor in TID order. + */ + while (!tuplesort_empty && (indexcursor == NULL || /* null on first time here */ + ItemPointerCompare(indexcursor, auxindexcursor) < 0)) + { + /* + * Get the next TID from the main sort. If it's empty, + * we set indexcursor to NULL. + */ + tuplesort_empty = !tuplesort_getdatum(main, true, + false, &ts_val, &ts_isnull, + NULL); + Assert(tuplesort_empty || !ts_isnull); + + if (!tuplesort_empty) + { + itemptr_decode(&decoded, DatumGetInt64(ts_val)); + indexcursor = &decoded; + } + else + { + indexcursor = NULL; + } + + CHECK_FOR_INTERRUPTS(); + } + + /* + * Now, if either: + * - the main sort is empty, or + * - indexcursor > auxindexcursor, + * + * then auxindexcursor identifies a TID that doesn't appear in + * the main sort. We likely need to insert it + * into the target index if it’s visible in the heap. + */ + if (tuplesort_empty || ItemPointerCompare(indexcursor, auxindexcursor) > 0) + { + tuplestore_putdatum(store, Int64GetDatum(itemptr_encode(auxindexcursor))); + num++; + } + } + } + + return num; +} + +typedef struct ValidateIndexScanState +{ + Tuplestorestate *store; + BlockNumber prev_block_number; + OffsetNumber prev_off_offset_number; +} ValidateIndexScanState; + +/* + * This is ReadStreamBlockNumberCB implementation which works as follows: + * + * 1) It iterates over a sorted tuplestore, where each element is an encoded + * ItemPointer + * + * 2) It returns the current BlockNumber and collects all OffsetNumbers + * for that block in per_buffer_data. + * + * 3) Once the code encounters a new BlockNumber, it stops reading more + * offsets and saves the OffsetNumber of the new block for the next call. + * + * 4) The list of offsets for a block is always terminated with InvalidOffsetNumber. + * + * This function is intended to be repeatedly called, each time returning + * the next block and its corresponding set of offsets. + */ +static BlockNumber +heapam_index_validate_scan_read_stream_next( + ReadStream *stream, + void *void_callback_private_data, + void *void_per_buffer_data + ) +{ + bool shoud_free; + Datum datum; + BlockNumber result = InvalidBlockNumber; + int i = 0; + + /* + * Retrieve the specialized callback state and the output buffer. + * callback_private_data keeps track of the previous block and offset + * from a prior invocation, if any. + */ + ValidateIndexScanState *callback_private_data = void_callback_private_data; + OffsetNumber *per_buffer_data = void_per_buffer_data; + + /* + * If there is a "leftover" offset number from the previous invocation, + * it means we had switched to a new block in the middle of the last call. + * We place that leftover offset number into the buffer first. + */ + if (callback_private_data->prev_off_offset_number != InvalidOffsetNumber) + { + Assert(callback_private_data->prev_block_number != InvalidBlockNumber); + /* + * 'result' is the block number to return. We set it to the block + * from the previous leftover offset. + */ + result = callback_private_data->prev_block_number; + /* Place leftover offset number in the output buffer. */ + per_buffer_data[i++] = callback_private_data->prev_off_offset_number; + /* + * Clear the leftover offset number so it won't be reused unless + * we encounter another block change. + */ + callback_private_data->prev_off_offset_number = InvalidOffsetNumber; + } + + /* + * Read from the tuplestore until we either run out of tuples or we + * encounter a block change. For each tuple: + * + * 1) Decode its block/offset from the Datum. + * 2) If it's the first time in this call (prev_block_number == InvalidBlockNumber), + * initialize prev_block_number. + * 3) If the block number matches the current block, collect the offset. + * 4) If the block number differs, save that offset as leftover and break + * so that the next call can handle the new block. + */ + while (tuplestore_getdatum(callback_private_data->store, true, &shoud_free, &datum)) + { + BlockNumber next_block_number; + ItemPointerData next_data; + + /* Decode the datum into an ItemPointer (block + offset). */ + itemptr_decode(&next_data, DatumGetInt64(datum)); + next_block_number = ItemPointerGetBlockNumber(&next_data); + + /* + * If we haven't set a block number yet this round, initialize it + * using the first tuple we read. + */ + if (callback_private_data->prev_block_number == InvalidBlockNumber) + callback_private_data->prev_block_number = next_block_number; + + /* + * Always set the result to be the "current" block number + * we are filling offsets for. + */ + result = callback_private_data->prev_block_number; + + /* + * If this tuple is from the same block, just store its offset + * in our per_buffer_data array. + */ + if (next_block_number == callback_private_data->prev_block_number) + { + per_buffer_data[i++] = ItemPointerGetOffsetNumber(&next_data); + + /* Free the datum if needed. */ + if (shoud_free) + pfree(DatumGetPointer(datum)); + } + else + { + /* + * If the block just changed, store the offset of the new block + * as leftover for the next invocation and break out. + */ + callback_private_data->prev_block_number = next_block_number; + callback_private_data->prev_off_offset_number = ItemPointerGetOffsetNumber(&next_data); + + /* Free the datum if needed. */ + if (shoud_free) + pfree(DatumGetPointer(datum)); + + /* Break to let the next call handle the new block. */ + break; + } + } + + /* + * Terminate the list of offsets for this block with an InvalidOffsetNumber. + */ + per_buffer_data[i] = InvalidOffsetNumber; + return result; +} + static void heapam_index_validate_scan(Relation heapRelation, Relation indexRelation, IndexInfo *indexInfo, Snapshot snapshot, - ValidateIndexState *state) + ValidateIndexState *state, + ValidateIndexState *auxState) { - TableScanDesc scan; - HeapScanDesc hscan; - HeapTuple heapTuple; Datum values[INDEX_MAX_KEYS]; bool isnull[INDEX_MAX_KEYS]; - ExprState *predicate; - TupleTableSlot *slot; - EState *estate; - ExprContext *econtext; - BlockNumber root_blkno = InvalidBlockNumber; - OffsetNumber root_offsets[MaxHeapTuplesPerPage]; - bool in_index[MaxHeapTuplesPerPage]; - BlockNumber previous_blkno = InvalidBlockNumber; - - /* state variables for the merge */ - ItemPointer indexcursor = NULL; - ItemPointerData decoded; - bool tuplesort_empty = false; + + TupleTableSlot *slot; + EState *estate; + ExprContext *econtext; + BufferAccessStrategy bstrategy = GetAccessStrategy(BAS_BULKREAD); + + int num_to_check; + Tuplestorestate *tuples_for_check; + ValidateIndexScanState callback_private_data; + + Buffer buf; + OffsetNumber* tuples; + ReadStream *read_stream; + + /* Use 10% of memory for tuple store. */ + int store_work_mem_part = maintenance_work_mem / 10; + + /* + * Encode TIDs as int8 values for the sort, rather than directly sorting + * item pointers. This can be significantly faster, primarily because TID + * is a pass-by-reference type on all platforms, whereas int8 is + * pass-by-value on most platforms. + */ + tuples_for_check = tuplestore_begin_datum(INT8OID, false, false, store_work_mem_part); /* * sanity checks */ Assert(OidIsValid(indexRelation->rd_rel->relam)); - /* - * Need an EState for evaluation of index expressions and partial-index - * predicates. Also a slot to hold the current tuple. - */ + num_to_check = heapam_index_validate_tuplesort_difference(state->tuplesort, + auxState->tuplesort, + tuples_for_check); + + /* It is our responsibility to close tuple sort as fast as we can */ + tuplesort_end(state->tuplesort); + tuplesort_end(auxState->tuplesort); + + state->tuplesort = auxState->tuplesort = NULL; + estate = CreateExecutorState(); econtext = GetPerTupleExprContext(estate); slot = MakeSingleTupleTableSlot(RelationGetDescr(heapRelation), - &TTSOpsHeapTuple); + &TTSOpsBufferHeapTuple); /* Arrange for econtext's scan tuple to be the tuple under test */ econtext->ecxt_scantuple = slot; - /* Set up execution state for predicate, if any. */ - predicate = ExecPrepareQual(indexInfo->ii_Predicate, estate); + callback_private_data.prev_block_number = InvalidBlockNumber; + callback_private_data.store = tuples_for_check; + callback_private_data.prev_off_offset_number = InvalidOffsetNumber; - /* - * Prepare for scan of the base relation. We need just those tuples - * satisfying the passed-in reference snapshot. We must disable syncscan - * here, because it's critical that we read from block zero forward to - * match the sorted TIDs. - */ - scan = table_beginscan_strat(heapRelation, /* relation */ - snapshot, /* snapshot */ - 0, /* number of keys */ - NULL, /* scan key */ - true, /* buffer access strategy OK */ - false); /* syncscan not OK */ - hscan = (HeapScanDesc) scan; + read_stream = read_stream_begin_relation(READ_STREAM_MAINTENANCE | READ_STREAM_USE_BATCHING, + bstrategy, + heapRelation, MAIN_FORKNUM, + heapam_index_validate_scan_read_stream_next, + &callback_private_data, + (MaxHeapTuplesPerPage + 1) * sizeof(OffsetNumber)); - pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL, - hscan->rs_nblocks); + pgstat_progress_update_param(PROGRESS_CREATEIDX_TUPLES_TOTAL, num_to_check); + pgstat_progress_update_param(PROGRESS_CREATEIDX_TUPLES_DONE, 0); - /* - * Scan all tuples matching the snapshot. - */ - while ((heapTuple = heap_getnext(scan, ForwardScanDirection)) != NULL) + while ((buf = read_stream_next_buffer(read_stream, (void*) &tuples)) != InvalidBuffer) { - ItemPointer heapcursor = &heapTuple->t_self; - ItemPointerData rootTuple; - OffsetNumber root_offnum; + HeapTupleData heap_tuple_data[MaxHeapTuplesPerPage]; + int i; + OffsetNumber off; + BlockNumber block_number; CHECK_FOR_INTERRUPTS(); - state->htups += 1; + LockBuffer(buf, BUFFER_LOCK_SHARE); + block_number = BufferGetBlockNumber(buf); - if ((previous_blkno == InvalidBlockNumber) || - (hscan->rs_cblock != previous_blkno)) + i = 0; + while ((off = tuples[i]) != InvalidOffsetNumber) { - pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE, - hscan->rs_cblock); - previous_blkno = hscan->rs_cblock; + ItemPointerData tid; + bool all_dead, found; + ItemPointerSet(&tid, block_number, off); + + found = heap_hot_search_buffer(&tid, heapRelation, buf, snapshot, + &heap_tuple_data[i], &all_dead, true); + if (!found) + ItemPointerSetInvalid(&heap_tuple_data[i].t_self); + i++; } + LockBuffer(buf, BUFFER_LOCK_UNLOCK); - /* - * As commented in table_index_build_scan, we should index heap-only - * tuples under the TIDs of their root tuples; so when we advance onto - * a new heap page, build a map of root item offsets on the page. - * - * This complicates merging against the tuplesort output: we will - * visit the live tuples in order by their offsets, but the root - * offsets that we need to compare against the index contents might be - * ordered differently. So we might have to "look back" within the - * tuplesort output, but only within the current page. We handle that - * by keeping a bool array in_index[] showing all the - * already-passed-over tuplesort output TIDs of the current page. We - * clear that array here, when advancing onto a new heap page. - */ - if (hscan->rs_cblock != root_blkno) + i = 0; + while ((off = tuples[i]) != InvalidOffsetNumber) { - Page page = BufferGetPage(hscan->rs_cbuf); - - LockBuffer(hscan->rs_cbuf, BUFFER_LOCK_SHARE); - heap_get_root_tuples(page, root_offsets); - LockBuffer(hscan->rs_cbuf, BUFFER_LOCK_UNLOCK); - - memset(in_index, 0, sizeof(in_index)); - - root_blkno = hscan->rs_cblock; - } - - /* Convert actual tuple TID to root TID */ - rootTuple = *heapcursor; - root_offnum = ItemPointerGetOffsetNumber(heapcursor); - - if (HeapTupleIsHeapOnly(heapTuple)) - { - root_offnum = root_offsets[root_offnum - 1]; - if (!OffsetNumberIsValid(root_offnum)) - ereport(ERROR, - (errcode(ERRCODE_DATA_CORRUPTED), - errmsg_internal("failed to find parent tuple for heap-only tuple at (%u,%u) in table \"%s\"", - ItemPointerGetBlockNumber(heapcursor), - ItemPointerGetOffsetNumber(heapcursor), - RelationGetRelationName(heapRelation)))); - ItemPointerSetOffsetNumber(&rootTuple, root_offnum); - } - - /* - * "merge" by skipping through the index tuples until we find or pass - * the current root tuple. - */ - while (!tuplesort_empty && - (!indexcursor || - ItemPointerCompare(indexcursor, &rootTuple) < 0)) - { - Datum ts_val; - bool ts_isnull; - - if (indexcursor) + if (ItemPointerIsValid(&heap_tuple_data[i].t_self)) { + ItemPointerData root_tid; + ItemPointerSet(&root_tid, block_number, off); + + /* Reset the per-tuple memory context for the next fetch. */ + MemoryContextReset(econtext->ecxt_per_tuple_memory); + ExecStoreBufferHeapTuple(&heap_tuple_data[i], slot, buf); + + /* Compute the key values and null flags for this tuple. */ + FormIndexDatum(indexInfo, + slot, + estate, + values, + isnull); + /* - * Remember index items seen earlier on the current heap page + * Insert the tuple into the target index. */ - if (ItemPointerGetBlockNumber(indexcursor) == root_blkno) - in_index[ItemPointerGetOffsetNumber(indexcursor) - 1] = true; + index_insert(indexRelation, + values, + isnull, + &root_tid, /* insert root tuple */ + heapRelation, + indexInfo->ii_Unique ? + UNIQUE_CHECK_YES : UNIQUE_CHECK_NO, + false, + indexInfo); } - tuplesort_empty = !tuplesort_getdatum(state->tuplesort, true, - false, &ts_val, &ts_isnull, - NULL); - Assert(tuplesort_empty || !ts_isnull); - if (!tuplesort_empty) - { - itemptr_decode(&decoded, DatumGetInt64(ts_val)); - indexcursor = &decoded; - } - else - { - /* Be tidy */ - indexcursor = NULL; - } + state->htups += 1; + pgstat_progress_incr_param(PROGRESS_CREATEIDX_TUPLES_DONE, 1); + i++; } - /* - * If the tuplesort has overshot *and* we didn't see a match earlier, - * then this tuple is missing from the index, so insert it. - */ - if ((tuplesort_empty || - ItemPointerCompare(indexcursor, &rootTuple) > 0) && - !in_index[root_offnum - 1]) - { - MemoryContextReset(econtext->ecxt_per_tuple_memory); - - /* Set up for predicate or expression evaluation */ - ExecStoreHeapTuple(heapTuple, slot, false); - - /* - * In a partial index, discard tuples that don't satisfy the - * predicate. - */ - if (predicate != NULL) - { - if (!ExecQual(predicate, econtext)) - continue; - } - - /* - * For the current heap tuple, extract all the attributes we use - * in this index, and note which are null. This also performs - * evaluation of any expressions needed. - */ - FormIndexDatum(indexInfo, - slot, - estate, - values, - isnull); - - /* - * You'd think we should go ahead and build the index tuple here, - * but some index AMs want to do further processing on the data - * first. So pass the values[] and isnull[] arrays, instead. - */ - - /* - * If the tuple is already committed dead, you might think we - * could suppress uniqueness checking, but this is no longer true - * in the presence of HOT, because the insert is actually a proxy - * for a uniqueness check on the whole HOT-chain. That is, the - * tuple we have here could be dead because it was already - * HOT-updated, and if so the updating transaction will not have - * thought it should insert index entries. The index AM will - * check the whole HOT-chain and correctly detect a conflict if - * there is one. - */ - - index_insert(indexRelation, - values, - isnull, - &rootTuple, - heapRelation, - indexInfo->ii_Unique ? - UNIQUE_CHECK_YES : UNIQUE_CHECK_NO, - false, - indexInfo); - - state->tups_inserted += 1; - } + ReleaseBuffer(buf); } - table_endscan(scan); - ExecDropSingleTupleTableSlot(slot); FreeExecutorState(estate); + read_stream_end(read_stream); + tuplestore_end(tuples_for_check); + /* These may have been pointing to the now-gone estate */ indexInfo->ii_ExpressionsState = NIL; indexInfo->ii_PredicateState = NULL; diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c index 8e509a51c11..6a4b348dd1b 100644 --- a/src/backend/catalog/index.c +++ b/src/backend/catalog/index.c @@ -714,11 +714,16 @@ UpdateIndexRelation(Oid indexoid, * already exists. * INDEX_CREATE_PARTITIONED: * create a partitioned index (table must be partitioned) + * INDEX_CREATE_AUXILIARY: + * mark index as auxiliary index * constr_flags: flags passed to index_constraint_create * (only if INDEX_CREATE_ADD_CONSTRAINT is set) * allow_system_table_mods: allow table to be a system catalog * is_internal: if true, post creation hook for new index * constraintId: if not NULL, receives OID of created constraint + * relpersistence: persistence level to use for index. In most of the + * cases it is should be equal to persistence level of table, + * auxiliary indexes are only exception here. * * Returns the OID of the created index. */ @@ -759,6 +764,7 @@ index_create(Relation heapRelation, bool invalid = (flags & INDEX_CREATE_INVALID) != 0; bool concurrent = (flags & INDEX_CREATE_CONCURRENT) != 0; bool partitioned = (flags & INDEX_CREATE_PARTITIONED) != 0; + bool auxiliary = (flags & INDEX_CREATE_AUXILIARY) != 0; char relkind; TransactionId relfrozenxid; MultiXactId relminmxid; @@ -784,7 +790,10 @@ index_create(Relation heapRelation, namespaceId = RelationGetNamespace(heapRelation); shared_relation = heapRelation->rd_rel->relisshared; mapped_relation = RelationIsMapped(heapRelation); - relpersistence = heapRelation->rd_rel->relpersistence; + if (auxiliary) + relpersistence = RELPERSISTENCE_UNLOGGED; /* aux indexes are always unlogged */ + else + relpersistence = heapRelation->rd_rel->relpersistence; /* * check parameters @@ -792,6 +801,11 @@ index_create(Relation heapRelation, if (indexInfo->ii_NumIndexAttrs < 1) elog(ERROR, "must index at least one column"); + if (indexInfo->ii_Am == STIR_AM_OID && !auxiliary) + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("user-defined indexes with STIR access method are not supported"))); + if (!allow_system_table_mods && IsSystemRelation(heapRelation) && IsNormalProcessingMode()) @@ -1397,7 +1411,8 @@ index_concurrently_create_copy(Relation heapRelation, Oid oldIndexId, false, /* not ready for inserts */ true, indexRelation->rd_indam->amsummarizing, - oldInfo->ii_WithoutOverlaps); + oldInfo->ii_WithoutOverlaps, + false); /* * Extract the list of column names and the column numbers for the new @@ -1472,6 +1487,154 @@ index_concurrently_create_copy(Relation heapRelation, Oid oldIndexId, return newIndexId; } +/* + * index_concurrently_create_aux + * + * Create concurrently an auxiliary index based on the definition of the one + * provided by caller. The index is inserted into catalogs and needs to be + * built later on. This is called during concurrent reindex processing. + * + * "tablespaceOid" is the tablespace to use for this index. + */ +Oid +index_concurrently_create_aux(Relation heapRelation, Oid mainIndexId, + Oid tablespaceOid, const char *newName) +{ + Relation indexRelation; + IndexInfo *oldInfo, + *newInfo; + Oid newIndexId = InvalidOid; + HeapTuple indexTuple; + + List *indexColNames = NIL; + List *indexExprs = NIL; + List *indexPreds = NIL; + + Oid *auxOpclassIds; + int16 *auxColoptions; + + indexRelation = index_open(mainIndexId, RowExclusiveLock); + + /* The new index needs some information from the old index */ + oldInfo = BuildIndexInfo(indexRelation); + + /* + * Build of an auxiliary index with exclusion constraints is not + * supported. + */ + if (oldInfo->ii_ExclusionOps != NULL) + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("auxiliary index creation for exclusion constraints is not supported"))); + + /* Get the array of class and column options IDs from index info */ + indexTuple = SearchSysCache1(INDEXRELID, ObjectIdGetDatum(mainIndexId)); + if (!HeapTupleIsValid(indexTuple)) + elog(ERROR, "cache lookup failed for index %u", mainIndexId); + + + /* + * Fetch the list of expressions and predicates directly from the + * catalogs. This cannot rely on the information from IndexInfo of the + * old index as these have been flattened for the planner. + */ + if (oldInfo->ii_Expressions != NIL) + { + Datum exprDatum; + char *exprString; + + exprDatum = SysCacheGetAttrNotNull(INDEXRELID, indexTuple, + Anum_pg_index_indexprs); + exprString = TextDatumGetCString(exprDatum); + indexExprs = (List *) stringToNode(exprString); + pfree(exprString); + } + if (oldInfo->ii_Predicate != NIL) + { + Datum predDatum; + char *predString; + + predDatum = SysCacheGetAttrNotNull(INDEXRELID, indexTuple, + Anum_pg_index_indpred); + predString = TextDatumGetCString(predDatum); + indexPreds = (List *) stringToNode(predString); + + /* Also convert to implicit-AND format */ + indexPreds = make_ands_implicit((Expr *) indexPreds); + pfree(predString); + } + + /* + * Build the index information for the new index. Note that rebuild of + * indexes with exclusion constraints is not supported, hence there is no + * need to fill all the ii_Exclusion* fields. + */ + newInfo = makeIndexInfo(oldInfo->ii_NumIndexAttrs, + oldInfo->ii_NumIndexKeyAttrs, + STIR_AM_OID, /* special AM for aux indexes */ + indexExprs, + indexPreds, + false, /* aux index are not unique */ + oldInfo->ii_NullsNotDistinct, + false, /* not ready for inserts */ + true, + false, /* aux are not summarizing */ + false, /* aux are not without overlaps */ + true /* auxiliary */); + + /* + * Extract the list of column names and the column numbers for the new + * index information. All this information will be used for the index + * creation. + */ + for (int i = 0; i < oldInfo->ii_NumIndexAttrs; i++) + { + TupleDesc indexTupDesc = RelationGetDescr(indexRelation); + Form_pg_attribute att = TupleDescAttr(indexTupDesc, i); + + indexColNames = lappend(indexColNames, NameStr(att->attname)); + newInfo->ii_IndexAttrNumbers[i] = oldInfo->ii_IndexAttrNumbers[i]; + } + + auxOpclassIds = palloc0(sizeof(Oid) * newInfo->ii_NumIndexAttrs); + auxColoptions = palloc0(sizeof(int16) * newInfo->ii_NumIndexAttrs); + + /* Fill with "any ops" */ + for (int i = 0; i < newInfo->ii_NumIndexAttrs; i++) + { + auxOpclassIds[i] = ANY_STIR_OPS_OID; + auxColoptions[i] = 0; + } + + newIndexId = index_create(heapRelation, + newName, + InvalidOid, /* indexRelationId */ + InvalidOid, /* parentIndexRelid */ + InvalidOid, /* parentConstraintId */ + InvalidRelFileNumber, /* relFileNumber */ + newInfo, + indexColNames, + STIR_AM_OID, + tablespaceOid, + indexRelation->rd_indcollation, + auxOpclassIds, + NULL, + auxColoptions, + NULL, + (Datum) 0, + INDEX_CREATE_SKIP_BUILD | INDEX_CREATE_CONCURRENT | INDEX_CREATE_AUXILIARY, + 0, + true, /* allow table to be a system catalog? */ + false, /* is_internal? */ + NULL); + + /* Close the relations used and clean up */ + index_close(indexRelation, NoLock); + ReleaseSysCache(indexTuple); + + return newIndexId; +} + /* * index_concurrently_build * @@ -2452,7 +2615,8 @@ BuildIndexInfo(Relation index) indexStruct->indisready, false, index->rd_indam->amsummarizing, - indexStruct->indisexclusion && indexStruct->indisunique); + indexStruct->indisexclusion && indexStruct->indisunique, + index->rd_rel->relam == STIR_AM_OID /* auxiliary iff STIR */); /* fill in attribute numbers */ for (i = 0; i < numAtts; i++) @@ -2512,7 +2676,8 @@ BuildDummyIndexInfo(Relation index) indexStruct->indisready, false, index->rd_indam->amsummarizing, - indexStruct->indisexclusion && indexStruct->indisunique); + indexStruct->indisexclusion && indexStruct->indisunique, + index->rd_rel->relam == STIR_AM_OID /* auxiliary iff STIR */); /* fill in attribute numbers */ for (i = 0; i < numAtts; i++) @@ -3288,12 +3453,21 @@ IndexCheckExclusion(Relation heapRelation, * * We do a concurrent index build by first inserting the catalog entry for the * index via index_create(), marking it not indisready and not indisvalid. + * Then we create special auxiliary index the same way. It based on STIR AM. * Then we commit our transaction and start a new one, then we wait for all * transactions that could have been modifying the table to terminate. Now - * we know that any subsequently-started transactions will see the index and + * we know that any subsequently-started transactions will see indexes and * honor its constraints on HOT updates; so while existing HOT-chains might * be broken with respect to the index, no currently live tuple will have an - * incompatible HOT update done to it. We now build the index normally via + * incompatible HOT update done to it. + * + * After we build auxiliary index. It is fast operation without any actual + * table scan. As result, we have empty STIR index. We commit transaction and + * again wait for all transactions that could have been modifying the table + * to terminate. At that moment all new tuples are going to be inserted into + * auxiliary index. + * + * We now build the index normally via * index_build(), while holding a weak lock that allows concurrent * insert/update/delete. Also, we index only tuples that are valid * as of the start of the scan (see table_index_build_scan), whereas a normal @@ -3303,14 +3477,17 @@ IndexCheckExclusion(Relation heapRelation, * bogus unique-index failures due to concurrent UPDATEs (we might see * different versions of the same row as being valid when we pass over them, * if we used HeapTupleSatisfiesVacuum). This leaves us with an index that - * does not contain any tuples added to the table while we built the index. + * does not contain any tuples added to the table while we built the index + * (but these tuples contained in auxiliary index). * * Next, we mark the index "indisready" (but still not "indisvalid") and - * commit the second transaction and start a third. Again we wait for all + * commit the third transaction and start a fourth. Again we wait for all * transactions that could have been modifying the table to terminate. Now * we know that any subsequently-started transactions will see the index and - * insert their new tuples into it. We then take a new reference snapshot - * which is passed to validate_index(). Any tuples that are valid according + * insert their new tuples into it. At the same moment we clear "indisready" for + * auxiliary index, since it is no more required to be updated. + * + * We then take a new reference snapshot, any tuples that are valid according * to this snap, but are not in the index, must be added to the index. * (Any tuples committed live after the snap will be inserted into the * index by their originating transaction. Any tuples committed dead before @@ -3318,12 +3495,14 @@ IndexCheckExclusion(Relation heapRelation, * that might care about them before we mark the index valid.) * * validate_index() works by first gathering all the TIDs currently in the - * index, using a bulkdelete callback that just stores the TIDs and doesn't + * indexes, using a bulkdelete callback that just stores the TIDs and doesn't * ever say "delete it". (This should be faster than a plain indexscan; * also, not all index AMs support full-index indexscan.) Then we sort the - * TIDs, and finally scan the table doing a "merge join" against the TID list - * to see which tuples are missing from the index. Thus we will ensure that - * all tuples valid according to the reference snapshot are in the index. + * TIDs of both auxiliary and target indexes, and doing a "merge join" against + * the TID lists to see which tuples from auxiliary index are missing from the + * target index. Thus we will ensure that all tuples valid according to the + * reference snapshot are in the index. Notice we need to do bulkdelete in the + * particular order: auxiliary first, target last. * * Building a unique index this way is tricky: we might try to insert a * tuple that is already dead or is in process of being deleted, and we @@ -3341,22 +3520,26 @@ IndexCheckExclusion(Relation heapRelation, * not index). Then we mark the index "indisvalid" and commit. Subsequent * transactions will be able to use it for queries. * - * Doing two full table scans is a brute-force strategy. We could try to be - * cleverer, eg storing new tuples in a special area of the table (perhaps - * making the table append-only by setting use_fsm). However that would - * add yet more locking issues. + * Also, some actions to concurrent drop the auxiliary index are performed. */ void -validate_index(Oid heapId, Oid indexId, Snapshot snapshot) +validate_index(Oid heapId, Oid indexId, Oid auxIndexId, Snapshot snapshot) { Relation heapRelation, - indexRelation; + indexRelation, + auxIndexRelation; IndexInfo *indexInfo; - IndexVacuumInfo ivinfo; - ValidateIndexState state; + IndexVacuumInfo ivinfo, auxivinfo; + ValidateIndexState state, auxState; Oid save_userid; int save_sec_context; int save_nestlevel; + /* Use 80% of maintenance_work_mem to target index sorting and + * 10% rest for auxiliary. + * + * Rest 10% will be used for tuplestore later. */ + int64 main_work_mem_part = (int64) maintenance_work_mem * 8 / 10; + int aux_work_mem_part = maintenance_work_mem / 10; { const int progress_index[] = { @@ -3389,6 +3572,7 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot) RestrictSearchPath(); indexRelation = index_open(indexId, RowExclusiveLock); + auxIndexRelation = index_open(auxIndexId, RowExclusiveLock); /* * Fetch info needed for index_insert. (You might think this should be @@ -3413,15 +3597,55 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot) ivinfo.strategy = NULL; ivinfo.validate_index = true; + /* + * Copy all info to auxiliary info, changing only relation. + */ + auxivinfo = ivinfo; + auxivinfo.index = auxIndexRelation; + /* * Encode TIDs as int8 values for the sort, rather than directly sorting * item pointers. This can be significantly faster, primarily because TID * is a pass-by-reference type on all platforms, whereas int8 is * pass-by-value on most platforms. */ + auxState.tuplesort = tuplesort_begin_datum(INT8OID, Int8LessOperator, + InvalidOid, false, + aux_work_mem_part, + NULL, TUPLESORT_NONE); + auxState.htups = auxState.itups = auxState.tups_inserted = 0; + + (void) index_bulk_delete(&auxivinfo, NULL, + validate_index_callback, &auxState); + /* If aux index is empty, merge may be skipped */ + if (auxState.itups == 0) + { + tuplesort_end(auxState.tuplesort); + auxState.tuplesort = NULL; + + /* Roll back any GUC changes executed by index functions */ + AtEOXact_GUC(false, save_nestlevel); + + /* Restore userid and security context */ + SetUserIdAndSecContext(save_userid, save_sec_context); + + /* Close rels, but keep locks */ + index_close(auxIndexRelation, NoLock); + index_close(indexRelation, NoLock); + table_close(heapRelation, NoLock); + + PushActiveSnapshot(GetTransactionSnapshot()); + limitXmin = GetActiveSnapshot()->xmin; + PopActiveSnapshot(); + InvalidateCatalogSnapshot(); + + Assert(!TransactionIdIsValid(MyProc->xmin)); + return limitXmin; + } + state.tuplesort = tuplesort_begin_datum(INT8OID, Int8LessOperator, InvalidOid, false, - maintenance_work_mem, + (int) main_work_mem_part, NULL, TUPLESORT_NONE); state.htups = state.itups = state.tups_inserted = 0; @@ -3444,27 +3668,30 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot) pgstat_progress_update_multi_param(3, progress_index, progress_vals); } tuplesort_performsort(state.tuplesort); + tuplesort_performsort(auxState.tuplesort); /* - * Now scan the heap and "merge" it with the index + * Now merge both indexes */ pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, - PROGRESS_CREATEIDX_PHASE_VALIDATE_TABLESCAN); + PROGRESS_CREATEIDX_PHASE_VALIDATE_IDXMERGE); table_index_validate_scan(heapRelation, indexRelation, indexInfo, snapshot, - &state); + &state, + &auxState); - /* Done with tuplesort object */ - tuplesort_end(state.tuplesort); + /* Tuple sort closed by table_index_validate_scan */ + Assert(state.tuplesort == NULL && auxState.tuplesort == NULL); /* Make sure to release resources cached in indexInfo (if needed). */ index_insert_cleanup(indexRelation, indexInfo); elog(DEBUG2, - "validate_index found %.0f heap tuples, %.0f index tuples; inserted %.0f missing tuples", - state.htups, state.itups, state.tups_inserted); + "validate_index fetched %.0f heap tuples, %.0f index tuples;" + " %.0f aux index tuples; inserted %.0f missing tuples", + state.htups, state.itups, auxState.itups, state.tups_inserted); /* Roll back any GUC changes executed by index functions */ AtEOXact_GUC(false, save_nestlevel); @@ -3473,6 +3700,7 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot) SetUserIdAndSecContext(save_userid, save_sec_context); /* Close rels, but keep locks */ + index_close(auxIndexRelation, NoLock); index_close(indexRelation, NoLock); table_close(heapRelation, NoLock); } @@ -3533,6 +3761,12 @@ index_set_state_flags(Oid indexId, IndexStateFlagsAction action) Assert(!indexForm->indisvalid); indexForm->indisvalid = true; break; + case INDEX_DROP_CLEAR_READY: + /* Clear indisready during a CREATE INDEX CONCURRENTLY sequence */ + Assert(indexForm->indisready); + Assert(!indexForm->indisvalid); + indexForm->indisready = false; + break; case INDEX_DROP_CLEAR_VALID: /* @@ -3804,6 +4038,13 @@ reindex_index(const ReindexStmt *stmt, Oid indexId, indexInfo->ii_ExclusionStrats = NULL; } + /* Auxiliary indexes are not allowed to be rebuilt */ + if (indexInfo->ii_Auxiliary) + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("reindex of auxiliary index \"%s\" not supported", + RelationGetRelationName(iRel)))); + /* Suppress use of the target index while rebuilding it */ SetReindexProcessing(heapId, indexId); @@ -4046,6 +4287,7 @@ reindex_relation(const ReindexStmt *stmt, Oid relid, int flags, { Oid indexOid = lfirst_oid(indexId); Oid indexNamespaceId = get_rel_namespace(indexOid); + Oid indexAm = get_rel_relam(indexOid); /* * Skip any invalid indexes on a TOAST table. These can only be @@ -4071,6 +4313,18 @@ reindex_relation(const ReindexStmt *stmt, Oid relid, int flags, continue; } + if (indexAm == STIR_AM_OID) + { + ereport(WARNING, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("skipping reindex of auxiliary index \"%s.%s\"", + get_namespace_name(indexNamespaceId), + get_rel_name(indexOid)))); + if (flags & REINDEX_REL_SUPPRESS_INDEX_USE) + RemoveReindexPending(indexOid); + continue; + } + reindex_index(stmt, indexOid, !(flags & REINDEX_REL_CHECK_CONSTRAINTS), persistence, params); diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql index 059e8778ca7..59b77ff7513 100644 --- a/src/backend/catalog/system_views.sql +++ b/src/backend/catalog/system_views.sql @@ -1308,16 +1308,17 @@ CREATE VIEW pg_stat_progress_create_index AS END AS command, CASE S.param10 WHEN 0 THEN 'initializing' WHEN 1 THEN 'waiting for writers before build' - WHEN 2 THEN 'building index' || + WHEN 2 THEN 'waiting for writers to use auxiliary index' + WHEN 3 THEN 'building index' || COALESCE((': ' || pg_indexam_progress_phasename(S.param9::oid, S.param11)), '') - WHEN 3 THEN 'waiting for writers before validation' - WHEN 4 THEN 'index validation: scanning index' - WHEN 5 THEN 'index validation: sorting tuples' - WHEN 6 THEN 'index validation: scanning table' - WHEN 7 THEN 'waiting for old snapshots' - WHEN 8 THEN 'waiting for readers before marking dead' - WHEN 9 THEN 'waiting for readers before dropping' + WHEN 4 THEN 'waiting for writers before validation' + WHEN 5 THEN 'index validation: scanning index' + WHEN 6 THEN 'index validation: sorting tuples' + WHEN 7 THEN 'index validation: merging indexes' + WHEN 8 THEN 'waiting for old snapshots' + WHEN 9 THEN 'waiting for readers before marking dead' + WHEN 10 THEN 'waiting for readers before dropping' END as phase, S.param4 AS lockers_total, S.param5 AS lockers_done, diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c index 974243c5c60..9c34825e97d 100644 --- a/src/backend/commands/indexcmds.c +++ b/src/backend/commands/indexcmds.c @@ -181,6 +181,7 @@ CheckIndexCompatible(Oid oldId, bool isWithoutOverlaps) { bool isconstraint; + bool isauxiliary; Oid *typeIds; Oid *collationIds; Oid *opclassIds; @@ -231,6 +232,7 @@ CheckIndexCompatible(Oid oldId, amcanorder = amRoutine->amcanorder; amsummarizing = amRoutine->amsummarizing; + isauxiliary = accessMethodId == STIR_AM_OID; /* * Compute the operator classes, collations, and exclusion operators for @@ -242,7 +244,8 @@ CheckIndexCompatible(Oid oldId, */ indexInfo = makeIndexInfo(numberOfAttributes, numberOfAttributes, accessMethodId, NIL, NIL, false, false, - false, false, amsummarizing, isWithoutOverlaps); + false, false, amsummarizing, + isWithoutOverlaps, isauxiliary); typeIds = palloc_array(Oid, numberOfAttributes); collationIds = palloc_array(Oid, numberOfAttributes); opclassIds = palloc_array(Oid, numberOfAttributes); @@ -552,6 +555,7 @@ DefineIndex(Oid tableId, { bool concurrent; char *indexRelationName; + char *auxIndexRelationName = NULL; char *accessMethodName; Oid *typeIds; Oid *collationIds; @@ -561,6 +565,7 @@ DefineIndex(Oid tableId, Oid namespaceId; Oid tablespaceId; Oid createdConstraintId = InvalidOid; + Oid auxIndexRelationId = InvalidOid; List *indexColNames; List *allIndexParams; Relation rel; @@ -582,6 +587,7 @@ DefineIndex(Oid tableId, int numberOfKeyAttributes; TransactionId limitXmin; ObjectAddress address; + ObjectAddress auxAddress; LockRelId heaprelid; LOCKTAG heaplocktag; LOCKMODE lockmode; @@ -832,6 +838,15 @@ DefineIndex(Oid tableId, stmt->excludeOpNames, stmt->primary, stmt->isconstraint); + /* + * Select name for auxiliary index + */ + if (concurrent) + auxIndexRelationName = ChooseRelationName(indexRelationName, + NULL, + "ccaux", + namespaceId, + false); /* * look up the access method, verify it can handle the requested features @@ -927,7 +942,8 @@ DefineIndex(Oid tableId, !concurrent, concurrent, amissummarizing, - stmt->iswithoutoverlaps); + stmt->iswithoutoverlaps, + false); typeIds = palloc_array(Oid, numberOfAttributes); collationIds = palloc_array(Oid, numberOfAttributes); @@ -1592,6 +1608,16 @@ DefineIndex(Oid tableId, return address; } + /* + * In case of concurrent build - create auxiliary index record. + */ + if (concurrent) + { + auxIndexRelationId = index_concurrently_create_aux(rel, indexRelationId, + tablespaceId, auxIndexRelationName); + ObjectAddressSet(auxAddress, RelationRelationId, auxIndexRelationId); + } + AtEOXact_GUC(false, root_save_nestlevel); SetUserIdAndSecContext(root_save_userid, root_save_sec_context); @@ -1620,11 +1646,11 @@ DefineIndex(Oid tableId, /* * For a concurrent build, it's important to make the catalog entries * visible to other transactions before we start to build the index. That - * will prevent them from making incompatible HOT updates. The new index - * will be marked not indisready and not indisvalid, so that no one else - * tries to either insert into it or use it for queries. + * will prevent them from making incompatible HOT updates. New indexes + * (main and auxiliary) will be marked not indisready and not indisvalid, + * so that no one else tries to either insert into it or use it for queries. * - * We must commit our current transaction so that the index becomes + * We must commit our current transaction so that the indexes becomes * visible; then start another. Note that all the data structures we just * built are lost in the commit. The only data we keep past here are the * relation IDs. @@ -1634,7 +1660,7 @@ DefineIndex(Oid tableId, * cannot block, even if someone else is waiting for access, because we * already have the same lock within our transaction. * - * Note: we don't currently bother with a session lock on the index, + * Note: we don't currently bother with a session lock on the indexes, * because there are no operations that could change its state while we * hold lock on the parent table. This might need to change later. */ @@ -1673,7 +1699,7 @@ DefineIndex(Oid tableId, * with the old list of indexes. Use ShareLock to consider running * transactions that hold locks that permit writing to the table. Note we * do not need to worry about xacts that open the table for writing after - * this point; they will see the new index when they open it. + * this point; they will see the new indexes when they open it. * * Note: the reason we use actual lock acquisition here, rather than just * checking the ProcArray and sleeping, is that deadlock is possible if @@ -1685,14 +1711,44 @@ DefineIndex(Oid tableId, /* * At this moment we are sure that there are no transactions with the - * table open for write that don't have this new index in their list of + * table open for write that don't have this new indexes in their list of * indexes. We have waited out all the existing transactions and any new - * transaction will have the new index in its list, but the index is still - * marked as "not-ready-for-inserts". The index is consulted while + * transaction will have both new indexes in its list, but indexes are still + * marked as "not-ready-for-inserts". The indexes are consulted while * deciding HOT-safety though. This arrangement ensures that no new HOT * chains can be created where the new tuple and the old tuple in the * chain have different index keys. * + * Now call build on auxiliary index. Index will be created empty without + * any actual heap scan, but marked as "ready-for-inserts". The goal of + * that index is accumulate new tuples while main index is actually built. + */ + + /* Set ActiveSnapshot since functions in the indexes may need it */ + PushActiveSnapshot(GetTransactionSnapshot()); + + index_concurrently_build(tableId, auxIndexRelationId); + /* we can do away with our snapshot */ + PopActiveSnapshot(); + + CommitTransactionCommand(); + StartTransactionCommand(); + + /* Tell concurrent index builds to ignore us, if index qualifies */ + if (safe_index) + set_indexsafe_procflags(); + pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, + PROGRESS_CREATEIDX_PHASE_WAIT_2); + /* + * Now we need to ensure are no transactions with the with auxiliary index + * marked as "not-ready-for-inserts". + */ + WaitForLockers(heaplocktag, ShareLock, true); + + /* + * At this moment we are sure what all new tuples in table are inserted into + * auxiliary index. Now it is time to build the target index itself. + * * We now take a new snapshot, and build the index using all tuples that * are visible in this snapshot. We can be sure that any HOT updates to * these tuples will be compatible with the index, since any updates made @@ -1727,9 +1783,28 @@ DefineIndex(Oid tableId, * the index marked as read-only for updates. */ pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, - PROGRESS_CREATEIDX_PHASE_WAIT_2); + PROGRESS_CREATEIDX_PHASE_WAIT_3); WaitForLockers(heaplocktag, ShareLock, true); + /* + * Updating pg_index might involve TOAST table access, so ensure we + * have a valid snapshot. + */ + PushActiveSnapshot(GetTransactionSnapshot()); + /* + * Now target index is marked as "ready" for all transactions. So, auxiliary + * index is not more needed. So, start removing process by reverting "ready" + * flag. + */ + index_set_state_flags(auxIndexRelationId, INDEX_DROP_CLEAR_READY); + PopActiveSnapshot(); + + CommitTransactionCommand(); + StartTransactionCommand(); + /* Tell concurrent index builds to ignore us, if index qualifies */ + if (safe_index) + set_indexsafe_procflags(); + /* * Now take the "reference snapshot" that will be used by validate_index() * to filter candidate tuples. Beware! There might still be snapshots in @@ -1747,24 +1822,14 @@ DefineIndex(Oid tableId, */ snapshot = RegisterSnapshot(GetTransactionSnapshot()); PushActiveSnapshot(snapshot); - /* - * Scan the index and the heap, insert any missing index entries. - */ - validate_index(tableId, indexRelationId, snapshot); - - /* - * Drop the reference snapshot. We must do this before waiting out other - * snapshot holders, else we will deadlock against other processes also - * doing CREATE INDEX CONCURRENTLY, which would see our snapshot as one - * they must wait for. But first, save the snapshot's xmin to use as - * limitXmin for GetCurrentVirtualXIDs(). + * Merge content of auxiliary and target indexes - insert any missing index entries. */ + validate_index(tableId, indexRelationId, auxIndexRelationId, snapshot); limitXmin = snapshot->xmin; PopActiveSnapshot(); UnregisterSnapshot(snapshot); - /* * The snapshot subsystem could still contain registered snapshots that * are holding back our process's advertised xmin; in particular, if @@ -1791,7 +1856,7 @@ DefineIndex(Oid tableId, */ INJECTION_POINT("define_index_before_set_valid", NULL); pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, - PROGRESS_CREATEIDX_PHASE_WAIT_3); + PROGRESS_CREATEIDX_PHASE_WAIT_4); WaitForOlderSnapshots(limitXmin, true); /* @@ -1816,6 +1881,53 @@ DefineIndex(Oid tableId, * to replan; so relcache flush on the index itself was sufficient.) */ CacheInvalidateRelcacheByRelid(heaprelid.relId); + CommitTransactionCommand(); + StartTransactionCommand(); + + /* Tell concurrent index builds to ignore us, if index qualifies */ + if (safe_index) + set_indexsafe_procflags(); + + pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, + PROGRESS_CREATEIDX_PHASE_WAIT_5); + /* Now wait for all transaction to see auxiliary as "non-ready for inserts" */ + WaitForLockers(heaplocktag, AccessExclusiveLock, true); + + CommitTransactionCommand(); + StartTransactionCommand(); + + /* + * Updating pg_index might involve TOAST table access, so ensure we + * have a valid snapshot. + */ + PushActiveSnapshot(GetTransactionSnapshot()); + /* Now it is time to mark auxiliary index as dead */ + index_concurrently_set_dead(tableId, auxIndexRelationId); + PopActiveSnapshot(); + + CommitTransactionCommand(); + StartTransactionCommand(); + + /* Tell concurrent index builds to ignore us, if index qualifies */ + if (safe_index) + set_indexsafe_procflags(); + + pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, + PROGRESS_CREATEIDX_PHASE_WAIT_6); + /* Now wait for all transaction to ignore auxiliary because it is dead */ + WaitForLockers(heaplocktag, AccessExclusiveLock, true); + + CommitTransactionCommand(); + StartTransactionCommand(); + + /* + * Drop auxiliary index. + * + * Use PERFORM_DELETION_CONCURRENT_LOCK so that index_drop() uses the + * right lock level. + */ + performDeletion(&auxAddress, DROP_RESTRICT, + PERFORM_DELETION_CONCURRENT_LOCK | PERFORM_DELETION_INTERNAL); /* * Last thing to do is release the session-level lock on the parent table. @@ -3570,6 +3682,7 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein typedef struct ReindexIndexInfo { Oid indexId; + Oid auxIndexId; Oid tableId; Oid amId; bool safe; /* for set_indexsafe_procflags */ @@ -3675,8 +3788,15 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein Oid cellOid = lfirst_oid(lc); Relation indexRelation = index_open(cellOid, ShareUpdateExclusiveLock); + IndexInfo* indexInfo = BuildDummyIndexInfo(indexRelation); - if (!indexRelation->rd_index->indisvalid) + + if (indexInfo->ii_Auxiliary) + ereport(WARNING,(errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("skipping reindex of auxiliary index \"%s.%s\"", + get_namespace_name(get_rel_namespace(cellOid)), + get_rel_name(cellOid)))); + else if (!indexRelation->rd_index->indisvalid) ereport(WARNING, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), errmsg("skipping reindex of invalid index \"%s.%s\"", @@ -3728,8 +3848,15 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein Oid cellOid = lfirst_oid(lc2); Relation indexRelation = index_open(cellOid, ShareUpdateExclusiveLock); + IndexInfo* indexInfo = BuildDummyIndexInfo(indexRelation); - if (!indexRelation->rd_index->indisvalid) + if (indexInfo->ii_Auxiliary) + ereport(WARNING, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("skipping reindex of auxiliary index \"%s.%s\"", + get_namespace_name(get_rel_namespace(cellOid)), + get_rel_name(cellOid)))); + else if (!indexRelation->rd_index->indisvalid) ereport(WARNING, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), errmsg("skipping reindex of invalid index \"%s.%s\"", @@ -3790,6 +3917,13 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("cannot reindex invalid index on TOAST table"))); + /* Auxiliary indexes are not allowed to be rebuilt */ + if (get_rel_relam(relationOid) == STIR_AM_OID) + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("reindex of auxiliary index \"%s\" not supported", + get_rel_name(relationOid)))); + /* * Check if parent relation can be locked and if it exists, * this needs to be done at this stage as the list of indexes @@ -3893,15 +4027,18 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein foreach(lc, indexIds) { char *concurrentName; + char *auxConcurrentName; ReindexIndexInfo *idx = lfirst(lc); ReindexIndexInfo *newidx; Oid newIndexId; + Oid auxIndexId; Relation indexRel; Relation heapRel; Oid save_userid; int save_sec_context; int save_nestlevel; Relation newIndexRel; + Relation auxIndexRel; LockRelId *lockrelid; Oid tablespaceid; @@ -3952,6 +4089,11 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein "ccnew", get_rel_namespace(indexRel->rd_index->indrelid), false); + auxConcurrentName = ChooseRelationName(get_rel_name(idx->indexId), + NULL, + "ccaux", + get_rel_namespace(indexRel->rd_index->indrelid), + false); /* Choose the new tablespace, indexes of toast tables are not moved */ if (OidIsValid(params->tablespaceOid) && @@ -3965,12 +4107,17 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein idx->indexId, tablespaceid, concurrentName); + auxIndexId = index_concurrently_create_aux(heapRel, + newIndexId, + tablespaceid, + auxConcurrentName); /* * Now open the relation of the new index, a session-level lock is * also needed on it. */ newIndexRel = index_open(newIndexId, ShareUpdateExclusiveLock); + auxIndexRel = index_open(auxIndexId, ShareUpdateExclusiveLock); /* * Save the list of OIDs and locks in private context @@ -3979,6 +4126,7 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein newidx = palloc_object(ReindexIndexInfo); newidx->indexId = newIndexId; + newidx->auxIndexId = auxIndexId; newidx->safe = idx->safe; newidx->tableId = idx->tableId; newidx->amId = idx->amId; @@ -3997,10 +4145,14 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein lockrelid = palloc_object(LockRelId); *lockrelid = newIndexRel->rd_lockInfo.lockRelId; relationLocks = lappend(relationLocks, lockrelid); + lockrelid = palloc_object(LockRelId); + *lockrelid = auxIndexRel->rd_lockInfo.lockRelId; + relationLocks = lappend(relationLocks, lockrelid); MemoryContextSwitchTo(oldcontext); index_close(indexRel, NoLock); + index_close(auxIndexRel, NoLock); index_close(newIndexRel, NoLock); /* Roll back any GUC changes executed by index functions */ @@ -4081,13 +4233,60 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein * doing that, wait until no running transactions could have the table of * the index open with the old list of indexes. See "phase 2" in * DefineIndex() for more details. + */ + + pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, + PROGRESS_CREATEIDX_PHASE_WAIT_1); + WaitForLockersMultiple(lockTags, ShareLock, true); + CommitTransactionCommand(); + + /* + * Now build all auxiliary indexes and mark them as "ready-for-inserts". + */ + foreach(lc, newIndexIds) + { + ReindexIndexInfo *newidx = lfirst(lc); + + StartTransactionCommand(); + + /* + * Check for user-requested abort. This is inside a transaction so as + * xact.c does not issue a useless WARNING, and ensures that + * session-level locks are cleaned up on abort. + */ + CHECK_FOR_INTERRUPTS(); + + /* Tell concurrent indexing to ignore us, if index qualifies */ + if (newidx->safe) + set_indexsafe_procflags(); + + /* Set ActiveSnapshot since functions in the indexes may need it */ + PushActiveSnapshot(GetTransactionSnapshot()); + + /* Build auxiliary index, it is fast - without any actual heap scan, just an empty index. */ + index_concurrently_build(newidx->tableId, newidx->auxIndexId); + + PopActiveSnapshot(); + CommitTransactionCommand(); + } + + StartTransactionCommand(); + + /* + * Because we don't take a snapshot in this transaction, there's no need + * to set the PROC_IN_SAFE_IC flag here. */ pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, - PROGRESS_CREATEIDX_PHASE_WAIT_1); + PROGRESS_CREATEIDX_PHASE_WAIT_2); + /* + * Wait until all auxiliary indexes are taken into account by all + * transactions. + */ WaitForLockersMultiple(lockTags, ShareLock, true); CommitTransactionCommand(); + /* Now it is time to perform target index build. */ foreach(lc, newIndexIds) { ReindexIndexInfo *newidx = lfirst(lc); @@ -4134,6 +4333,41 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein * need to set the PROC_IN_SAFE_IC flag here. */ + pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, + PROGRESS_CREATEIDX_PHASE_WAIT_3); + WaitForLockersMultiple(lockTags, ShareLock, true); + CommitTransactionCommand(); + + /* + * At this moment all target indexes are marked as "ready-to-insert". So, + * we are free to start process of dropping auxiliary indexes. + */ + foreach(lc, newIndexIds) + { + ReindexIndexInfo *newidx = lfirst(lc); + StartTransactionCommand(); + /* + * Check for user-requested abort. This is inside a transaction so as + * xact.c does not issue a useless WARNING, and ensures that + * session-level locks are cleaned up on abort. + */ + CHECK_FOR_INTERRUPTS(); + + /* Tell concurrent indexing to ignore us, if index qualifies */ + if (newidx->safe) + set_indexsafe_procflags(); + + /* + * Updating pg_index might involve TOAST table access, so ensure we + * have a valid snapshot. + */ + PushActiveSnapshot(GetTransactionSnapshot()); + index_set_state_flags(newidx->auxIndexId, INDEX_DROP_CLEAR_READY); + PopActiveSnapshot(); + + CommitTransactionCommand(); + } + /* * Phase 3 of REINDEX CONCURRENTLY * @@ -4141,12 +4375,6 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein * were created during the previous phase. See "phase 3" in DefineIndex() * for more details. */ - - pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, - PROGRESS_CREATEIDX_PHASE_WAIT_2); - WaitForLockersMultiple(lockTags, ShareLock, true); - CommitTransactionCommand(); - foreach(lc, newIndexIds) { ReindexIndexInfo *newidx = lfirst(lc); @@ -4184,7 +4412,7 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein progress_vals[3] = newidx->amId; pgstat_progress_update_multi_param(4, progress_index, progress_vals); - validate_index(newidx->tableId, newidx->indexId, snapshot); + validate_index(newidx->tableId, newidx->indexId, newidx->auxIndexId, snapshot); /* * We can now do away with our active snapshot, we still need to save @@ -4213,7 +4441,7 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein * there's no need to set the PROC_IN_SAFE_IC flag here. */ pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, - PROGRESS_CREATEIDX_PHASE_WAIT_3); + PROGRESS_CREATEIDX_PHASE_WAIT_4); WaitForOlderSnapshots(limitXmin, true); CommitTransactionCommand(); @@ -4303,14 +4531,14 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein /* * Phase 5 of REINDEX CONCURRENTLY * - * Mark the old indexes as dead. First we must wait until no running - * transaction could be using the index for a query. See also + * Mark the old and auxiliary indexes as dead. First we must wait until no + * running transaction could be using the index for a query. See also * index_drop() for more details. */ INJECTION_POINT("reindex_relation_concurrently_before_set_dead", NULL); pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, - PROGRESS_CREATEIDX_PHASE_WAIT_4); + PROGRESS_CREATEIDX_PHASE_WAIT_5); WaitForLockersMultiple(lockTags, AccessExclusiveLock, true); foreach(lc, indexIds) @@ -4335,6 +4563,28 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein PopActiveSnapshot(); } + foreach(lc, newIndexIds) + { + ReindexIndexInfo *newidx = lfirst(lc); + + /* + * Check for user-requested abort. This is inside a transaction so as + * xact.c does not issue a useless WARNING, and ensures that + * session-level locks are cleaned up on abort. + */ + CHECK_FOR_INTERRUPTS(); + + /* + * Updating pg_index might involve TOAST table access, so ensure we + * have a valid snapshot. + */ + PushActiveSnapshot(GetTransactionSnapshot()); + + index_concurrently_set_dead(newidx->tableId, newidx->auxIndexId); + + PopActiveSnapshot(); + } + /* Commit this transaction to make the updates visible. */ CommitTransactionCommand(); StartTransactionCommand(); @@ -4348,11 +4598,11 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein /* * Phase 6 of REINDEX CONCURRENTLY * - * Drop the old indexes. + * Drop the old and auxiliary indexes. */ pgstat_progress_update_param(PROGRESS_CREATEIDX_PHASE, - PROGRESS_CREATEIDX_PHASE_WAIT_5); + PROGRESS_CREATEIDX_PHASE_WAIT_6); WaitForLockersMultiple(lockTags, AccessExclusiveLock, true); PushActiveSnapshot(GetTransactionSnapshot()); @@ -4372,6 +4622,18 @@ ReindexRelationConcurrently(const ReindexStmt *stmt, Oid relationOid, const Rein add_exact_object_address(&object, objects); } + foreach(lc, newIndexIds) + { + ReindexIndexInfo *idx = lfirst(lc); + ObjectAddress object; + + object.classId = RelationRelationId; + object.objectId = idx->auxIndexId; + object.objectSubId = 0; + + add_exact_object_address(&object, objects); + } + /* * Use PERFORM_DELETION_CONCURRENT_LOCK so that index_drop() uses the * right lock level. diff --git a/src/backend/nodes/makefuncs.c b/src/backend/nodes/makefuncs.c index e97e0943f5b..b556ba4817b 100644 --- a/src/backend/nodes/makefuncs.c +++ b/src/backend/nodes/makefuncs.c @@ -834,7 +834,7 @@ IndexInfo * makeIndexInfo(int numattrs, int numkeyattrs, Oid amoid, List *expressions, List *predicates, bool unique, bool nulls_not_distinct, bool isready, bool concurrent, bool summarizing, - bool withoutoverlaps) + bool withoutoverlaps, bool auxiliary) { IndexInfo *n = makeNode(IndexInfo); @@ -850,6 +850,7 @@ makeIndexInfo(int numattrs, int numkeyattrs, Oid amoid, List *expressions, n->ii_Concurrent = concurrent; n->ii_Summarizing = summarizing; n->ii_WithoutOverlaps = withoutoverlaps; + n->ii_Auxiliary = auxiliary; /* summarizing indexes cannot contain non-key attributes */ Assert(!summarizing || (numkeyattrs == numattrs)); @@ -875,7 +876,6 @@ makeIndexInfo(int numattrs, int numkeyattrs, Oid amoid, List *expressions, /* initialize index-build state to default */ n->ii_BrokenHotChain = false; n->ii_ParallelWorkers = 0; - n->ii_Auxiliary = false; /* set up for possible use by index AM */ n->ii_Am = amoid; diff --git a/src/include/access/tableam.h b/src/include/access/tableam.h index e16bf025692..22446b32157 100644 --- a/src/include/access/tableam.h +++ b/src/include/access/tableam.h @@ -706,7 +706,8 @@ typedef struct TableAmRoutine Relation index_rel, IndexInfo *index_info, Snapshot snapshot, - ValidateIndexState *state); + ValidateIndexState *state, + ValidateIndexState *aux_state); /* ------------------------------------------------------------------------ @@ -1803,19 +1804,24 @@ table_index_build_range_scan(Relation table_rel, * table_index_validate_scan - second table scan for concurrent index build * * See validate_index() for an explanation. + * + * Note: it is responsibility of that function to close sortstates in + * both `state` and `auxstate`. */ static inline void table_index_validate_scan(Relation table_rel, Relation index_rel, IndexInfo *index_info, Snapshot snapshot, - ValidateIndexState *state) + ValidateIndexState *state, + ValidateIndexState *auxstate) { table_rel->rd_tableam->index_validate_scan(table_rel, index_rel, index_info, snapshot, - state); + state, + auxstate); } diff --git a/src/include/catalog/index.h b/src/include/catalog/index.h index dda95e54903..c29f44f2465 100644 --- a/src/include/catalog/index.h +++ b/src/include/catalog/index.h @@ -25,6 +25,7 @@ typedef enum { INDEX_CREATE_SET_READY, INDEX_CREATE_SET_VALID, + INDEX_DROP_CLEAR_READY, INDEX_DROP_CLEAR_VALID, INDEX_DROP_SET_DEAD, } IndexStateFlagsAction; @@ -65,6 +66,7 @@ extern void index_check_primary_key(Relation heapRel, #define INDEX_CREATE_IF_NOT_EXISTS (1 << 4) #define INDEX_CREATE_PARTITIONED (1 << 5) #define INDEX_CREATE_INVALID (1 << 6) +#define INDEX_CREATE_AUXILIARY (1 << 7) extern Oid index_create(Relation heapRelation, const char *indexRelationName, @@ -100,6 +102,11 @@ extern Oid index_concurrently_create_copy(Relation heapRelation, Oid tablespaceOid, const char *newName); +extern Oid index_concurrently_create_aux(Relation heapRelation, + Oid mainIndexId, + Oid tablespaceOid, + const char *newName); + extern void index_concurrently_build(Oid heapRelationId, Oid indexRelationId); @@ -145,7 +152,7 @@ extern void index_build(Relation heapRelation, bool isreindex, bool parallel); -extern void validate_index(Oid heapId, Oid indexId, Snapshot snapshot); +extern void validate_index(Oid heapId, Oid indexId, Oid auxIndexId, Snapshot snapshot); extern void index_set_state_flags(Oid indexId, IndexStateFlagsAction action); diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h index 1cde4bd9bcf..9e93a4d9310 100644 --- a/src/include/commands/progress.h +++ b/src/include/commands/progress.h @@ -94,14 +94,15 @@ /* Phases of CREATE INDEX (as advertised via PROGRESS_CREATEIDX_PHASE) */ #define PROGRESS_CREATEIDX_PHASE_WAIT_1 1 -#define PROGRESS_CREATEIDX_PHASE_BUILD 2 -#define PROGRESS_CREATEIDX_PHASE_WAIT_2 3 -#define PROGRESS_CREATEIDX_PHASE_VALIDATE_IDXSCAN 4 -#define PROGRESS_CREATEIDX_PHASE_VALIDATE_SORT 5 -#define PROGRESS_CREATEIDX_PHASE_VALIDATE_TABLESCAN 6 -#define PROGRESS_CREATEIDX_PHASE_WAIT_3 7 +#define PROGRESS_CREATEIDX_PHASE_WAIT_2 2 +#define PROGRESS_CREATEIDX_PHASE_BUILD 3 +#define PROGRESS_CREATEIDX_PHASE_WAIT_3 4 +#define PROGRESS_CREATEIDX_PHASE_VALIDATE_IDXSCAN 5 +#define PROGRESS_CREATEIDX_PHASE_VALIDATE_SORT 6 +#define PROGRESS_CREATEIDX_PHASE_VALIDATE_IDXMERGE 7 #define PROGRESS_CREATEIDX_PHASE_WAIT_4 8 #define PROGRESS_CREATEIDX_PHASE_WAIT_5 9 +#define PROGRESS_CREATEIDX_PHASE_WAIT_6 10 /* * Subphases of CREATE INDEX, for index_build. diff --git a/src/include/nodes/makefuncs.h b/src/include/nodes/makefuncs.h index 5473ce9a288..4904748f5fc 100644 --- a/src/include/nodes/makefuncs.h +++ b/src/include/nodes/makefuncs.h @@ -99,7 +99,8 @@ extern IndexInfo *makeIndexInfo(int numattrs, int numkeyattrs, Oid amoid, List *expressions, List *predicates, bool unique, bool nulls_not_distinct, bool isready, bool concurrent, - bool summarizing, bool withoutoverlaps); + bool summarizing, bool withoutoverlaps, + bool auxiliary); extern Node *makeStringConst(char *str, int location); extern DefElem *makeDefElem(char *name, Node *arg, int location); diff --git a/src/test/regress/expected/create_index.out b/src/test/regress/expected/create_index.out index c743fc769cb..aa4fa76358a 100644 --- a/src/test/regress/expected/create_index.out +++ b/src/test/regress/expected/create_index.out @@ -1423,6 +1423,7 @@ DETAIL: Key (f1)=(b) already exists. CREATE UNIQUE INDEX CONCURRENTLY concur_index3 ON concur_heap(f2); ERROR: could not create unique index "concur_index3" DETAIL: Key (f2)=(b) is duplicated. +DROP INDEX concur_index3_ccaux; -- test that expression indexes and partial indexes work concurrently CREATE INDEX CONCURRENTLY concur_index4 on concur_heap(f2) WHERE f1='a'; CREATE INDEX CONCURRENTLY concur_index5 on concur_heap(f2) WHERE f1='x'; @@ -3197,6 +3198,7 @@ INSERT INTO concur_reindex_tab4 VALUES (1), (1), (2); CREATE UNIQUE INDEX CONCURRENTLY concur_reindex_ind5 ON concur_reindex_tab4 (c1); ERROR: could not create unique index "concur_reindex_ind5" DETAIL: Key (c1)=(1) is duplicated. +DROP INDEX concur_reindex_ind5_ccaux; -- Reindexing concurrently this index fails with the same failure. -- The extra index created is itself invalid, and can be dropped. REINDEX INDEX CONCURRENTLY concur_reindex_ind5; @@ -3209,8 +3211,10 @@ DETAIL: Key (c1)=(1) is duplicated. c1 | integer | | | Indexes: "concur_reindex_ind5" UNIQUE, btree (c1) INVALID + "concur_reindex_ind5_ccaux" stir (c1) INVALID "concur_reindex_ind5_ccnew" UNIQUE, btree (c1) INVALID +DROP INDEX concur_reindex_ind5_ccaux; DROP INDEX concur_reindex_ind5_ccnew; -- This makes the previous failure go away, so the index can become valid. DELETE FROM concur_reindex_tab4 WHERE c1 = 1; @@ -3238,6 +3242,44 @@ Indexes: "concur_reindex_ind5" UNIQUE, btree (c1) DROP TABLE concur_reindex_tab4; +-- Check handling of auxiliary indexes +CREATE TABLE aux_index_tab5 (c1 int); +INSERT INTO aux_index_tab5 VALUES (1), (1), (2); +-- This trick creates an invalid index and auxiliary index for it +CREATE UNIQUE INDEX CONCURRENTLY aux_index_ind6 ON aux_index_tab5 (c1); +ERROR: could not create unique index "aux_index_ind6" +DETAIL: Key (c1)=(1) is duplicated. +\d aux_index_tab5 + Table "public.aux_index_tab5" + Column | Type | Collation | Nullable | Default +--------+---------+-----------+----------+--------- + c1 | integer | | | +Indexes: + "aux_index_ind6" UNIQUE, btree (c1) INVALID + "aux_index_ind6_ccaux" stir (c1) INVALID + +-- Not allowed to reindex auxiliary index +REINDEX INDEX aux_index_ind6_ccaux; +ERROR: reindex of auxiliary index "aux_index_ind6_ccaux" not supported +-- Concurrently also +REINDEX INDEX CONCURRENTLY aux_index_ind6_ccaux; +ERROR: reindex of auxiliary index "aux_index_ind6_ccaux" not supported +-- This makes the previous failure go away, so the index can become valid. +DELETE FROM concur_reindex_tab4 WHERE c1 = 1; +ERROR: relation "concur_reindex_tab4" does not exist +LINE 1: DELETE FROM concur_reindex_tab4 WHERE c1 = 1; + ^ +-- Should be skipped during reindex +REINDEX TABLE aux_index_tab5; +ERROR: could not create unique index "aux_index_ind6" +DETAIL: Key (c1)=(1) is duplicated. +-- Should be skipped during concurrent reindex +REINDEX TABLE CONCURRENTLY aux_index_tab5; +WARNING: skipping reindex of invalid index "public.aux_index_ind6" +HINT: Use DROP INDEX or REINDEX INDEX. +WARNING: skipping reindex of auxiliary index "public.aux_index_ind6_ccaux" +NOTICE: table "aux_index_tab5" has no indexes that can be reindexed concurrently +DROP TABLE aux_index_tab5; -- Check handling of indexes with expressions and predicates. The -- definitions of the rebuilt indexes should match the original -- definitions. diff --git a/src/test/regress/expected/indexing.out b/src/test/regress/expected/indexing.out index 4d29fb85293..54b251b96ea 100644 --- a/src/test/regress/expected/indexing.out +++ b/src/test/regress/expected/indexing.out @@ -1585,10 +1585,11 @@ select indexrelid::regclass, indisvalid, --------------------------------+------------+-----------------------+------------------------------- parted_isvalid_idx | f | parted_isvalid_tab | parted_isvalid_idx_11 | f | parted_isvalid_tab_11 | parted_isvalid_tab_1_expr_idx + parted_isvalid_idx_11_ccaux | f | parted_isvalid_tab_11 | parted_isvalid_tab_12_expr_idx | t | parted_isvalid_tab_12 | parted_isvalid_tab_1_expr_idx parted_isvalid_tab_1_expr_idx | f | parted_isvalid_tab_1 | parted_isvalid_idx parted_isvalid_tab_2_expr_idx | t | parted_isvalid_tab_2 | parted_isvalid_idx -(5 rows) +(6 rows) drop table parted_isvalid_tab; -- Check state of replica indexes when attaching a partition. diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out index 7c52181cbcb..917e4b208f8 100644 --- a/src/test/regress/expected/rules.out +++ b/src/test/regress/expected/rules.out @@ -2060,14 +2060,15 @@ pg_stat_progress_create_index| SELECT s.pid, CASE s.param10 WHEN 0 THEN 'initializing'::text WHEN 1 THEN 'waiting for writers before build'::text - WHEN 2 THEN ('building index'::text || COALESCE((': '::text || pg_indexam_progress_phasename((s.param9)::oid, s.param11)), ''::text)) - WHEN 3 THEN 'waiting for writers before validation'::text - WHEN 4 THEN 'index validation: scanning index'::text - WHEN 5 THEN 'index validation: sorting tuples'::text - WHEN 6 THEN 'index validation: scanning table'::text - WHEN 7 THEN 'waiting for old snapshots'::text - WHEN 8 THEN 'waiting for readers before marking dead'::text - WHEN 9 THEN 'waiting for readers before dropping'::text + WHEN 2 THEN 'waiting for writers to use auxiliary index'::text + WHEN 3 THEN ('building index'::text || COALESCE((': '::text || pg_indexam_progress_phasename((s.param9)::oid, s.param11)), ''::text)) + WHEN 4 THEN 'waiting for writers before validation'::text + WHEN 5 THEN 'index validation: scanning index'::text + WHEN 6 THEN 'index validation: sorting tuples'::text + WHEN 7 THEN 'index validation: merging indexes'::text + WHEN 8 THEN 'waiting for old snapshots'::text + WHEN 9 THEN 'waiting for readers before marking dead'::text + WHEN 10 THEN 'waiting for readers before dropping'::text ELSE NULL::text END AS phase, s.param4 AS lockers_total, diff --git a/src/test/regress/sql/create_index.sql b/src/test/regress/sql/create_index.sql index eabc9623b20..7ae8e44019b 100644 --- a/src/test/regress/sql/create_index.sql +++ b/src/test/regress/sql/create_index.sql @@ -499,6 +499,7 @@ CREATE UNIQUE INDEX CONCURRENTLY IF NOT EXISTS concur_index2 ON concur_heap(f1); INSERT INTO concur_heap VALUES ('b','x'); -- check if constraint is enforced properly at build time CREATE UNIQUE INDEX CONCURRENTLY concur_index3 ON concur_heap(f2); +DROP INDEX concur_index3_ccaux; -- test that expression indexes and partial indexes work concurrently CREATE INDEX CONCURRENTLY concur_index4 on concur_heap(f2) WHERE f1='a'; CREATE INDEX CONCURRENTLY concur_index5 on concur_heap(f2) WHERE f1='x'; @@ -1311,10 +1312,12 @@ CREATE TABLE concur_reindex_tab4 (c1 int); INSERT INTO concur_reindex_tab4 VALUES (1), (1), (2); -- This trick creates an invalid index. CREATE UNIQUE INDEX CONCURRENTLY concur_reindex_ind5 ON concur_reindex_tab4 (c1); +DROP INDEX concur_reindex_ind5_ccaux; -- Reindexing concurrently this index fails with the same failure. -- The extra index created is itself invalid, and can be dropped. REINDEX INDEX CONCURRENTLY concur_reindex_ind5; \d concur_reindex_tab4 +DROP INDEX concur_reindex_ind5_ccaux; DROP INDEX concur_reindex_ind5_ccnew; -- This makes the previous failure go away, so the index can become valid. DELETE FROM concur_reindex_tab4 WHERE c1 = 1; @@ -1326,6 +1329,24 @@ REINDEX INDEX CONCURRENTLY concur_reindex_ind5; \d concur_reindex_tab4 DROP TABLE concur_reindex_tab4; +-- Check handling of auxiliary indexes +CREATE TABLE aux_index_tab5 (c1 int); +INSERT INTO aux_index_tab5 VALUES (1), (1), (2); +-- This trick creates an invalid index and auxiliary index for it +CREATE UNIQUE INDEX CONCURRENTLY aux_index_ind6 ON aux_index_tab5 (c1); +\d aux_index_tab5 +-- Not allowed to reindex auxiliary index +REINDEX INDEX aux_index_ind6_ccaux; +-- Concurrently also +REINDEX INDEX CONCURRENTLY aux_index_ind6_ccaux; +-- This makes the previous failure go away, so the index can become valid. +DELETE FROM concur_reindex_tab4 WHERE c1 = 1; +-- Should be skipped during reindex +REINDEX TABLE aux_index_tab5; +-- Should be skipped during concurrent reindex +REINDEX TABLE CONCURRENTLY aux_index_tab5; +DROP TABLE aux_index_tab5; + -- Check handling of indexes with expressions and predicates. The -- definitions of the rebuilt indexes should match the original -- definitions. -- 2.43.0