mirror of
				https://github.com/postgres/postgres.git
				synced 2025-10-25 13:17:41 +03:00 
			
		
		
		
	Assert lack of hazardous buffer locks before possible catalog read.
Commit0bada39c83fixed a bug of this kind, which existed in all branches for six days before detection. While the probability of reaching the trouble was low, the disruption was extreme. No new backends could start, and service restoration needed an immediate shutdown. Hence, add this to catch the next bug like it. The new check in RelationIdGetRelation() suffices to make autovacuum detect the bug in commit243e9b40f1that led to commit0bada39. This also checks in a number of similar places. It replaces each Assert(IsTransactionState()) that pertained to a conditional catalog read. No back-patch for now, but a back-patch of commit243e9b4should back-patch this, too. A back-patch could omit the src/test/regress changes, since back branches won't gain new index columns. Reported-by: Alexander Lakhin <exclusion@gmail.com> Discussion: https://postgr.es/m/20250410191830.0e.nmisch@google.com Discussion: https://postgr.es/m/10ec0bc3-5933-1189-6bb8-5dec4114558e@gmail.com
This commit is contained in:
		| @@ -34,6 +34,7 @@ | ||||
| #include "catalog/pg_namespace.h" | ||||
| #include "catalog/pg_parameter_acl.h" | ||||
| #include "catalog/pg_replication_origin.h" | ||||
| #include "catalog/pg_seclabel.h" | ||||
| #include "catalog/pg_shdepend.h" | ||||
| #include "catalog/pg_shdescription.h" | ||||
| #include "catalog/pg_shseclabel.h" | ||||
| @@ -135,6 +136,36 @@ IsCatalogRelationOid(Oid relid) | ||||
| 	return (relid < (Oid) FirstUnpinnedObjectId); | ||||
| } | ||||
|  | ||||
| /* | ||||
|  * IsCatalogTextUniqueIndexOid | ||||
|  *		True iff the relation identified by this OID is a catalog UNIQUE index | ||||
|  *		having a column of type "text". | ||||
|  * | ||||
|  *		The relcache must not use these indexes.  Inserting into any UNIQUE | ||||
|  *		index compares index keys while holding BUFFER_LOCK_EXCLUSIVE. | ||||
|  *		bttextcmp() can search the COLLID catcache.  Depending on concurrent | ||||
|  *		invalidation traffic, catcache can reach relcache builds.  A backend | ||||
|  *		would self-deadlock on LWLocks if the relcache build read the | ||||
|  *		exclusive-locked buffer. | ||||
|  * | ||||
|  *		To avoid being itself the cause of self-deadlock, this doesn't read | ||||
|  *		catalogs.  Instead, it uses a hard-coded list with a supporting | ||||
|  *		regression test. | ||||
|  */ | ||||
| bool | ||||
| IsCatalogTextUniqueIndexOid(Oid relid) | ||||
| { | ||||
| 	switch (relid) | ||||
| 	{ | ||||
| 		case ParameterAclParnameIndexId: | ||||
| 		case ReplicationOriginNameIndex: | ||||
| 		case SecLabelObjectIndexId: | ||||
| 		case SharedSecLabelObjectIndexId: | ||||
| 			return true; | ||||
| 	} | ||||
| 	return false; | ||||
| } | ||||
|  | ||||
| /* | ||||
|  * IsInplaceUpdateRelation | ||||
|  *		True iff core code performs inplace updates on the relation. | ||||
|   | ||||
| @@ -40,6 +40,9 @@ | ||||
| #include "access/tableam.h" | ||||
| #include "access/xloginsert.h" | ||||
| #include "access/xlogutils.h" | ||||
| #ifdef USE_ASSERT_CHECKING | ||||
| #include "catalog/pg_tablespace_d.h" | ||||
| #endif | ||||
| #include "catalog/storage.h" | ||||
| #include "catalog/storage_xlog.h" | ||||
| #include "executor/instrument.h" | ||||
| @@ -541,6 +544,10 @@ static void RelationCopyStorageUsingBuffer(RelFileLocator srclocator, | ||||
| 										   ForkNumber forkNum, bool permanent); | ||||
| static void AtProcExit_Buffers(int code, Datum arg); | ||||
| static void CheckForBufferLeaks(void); | ||||
| #ifdef USE_ASSERT_CHECKING | ||||
| static void AssertNotCatalogBufferLock(LWLock *lock, LWLockMode mode, | ||||
| 									   void *unused_context); | ||||
| #endif | ||||
| static int	rlocator_comparator(const void *p1, const void *p2); | ||||
| static inline int buffertag_comparator(const BufferTag *ba, const BufferTag *bb); | ||||
| static inline int ckpt_buforder_comparator(const CkptSortItem *a, const CkptSortItem *b); | ||||
| @@ -4097,6 +4104,69 @@ CheckForBufferLeaks(void) | ||||
| #endif | ||||
| } | ||||
|  | ||||
| #ifdef USE_ASSERT_CHECKING | ||||
| /* | ||||
|  * Check for exclusive-locked catalog buffers.  This is the core of | ||||
|  * AssertCouldGetRelation(). | ||||
|  * | ||||
|  * A backend would self-deadlock on LWLocks if the catalog scan read the | ||||
|  * exclusive-locked buffer.  The main threat is exclusive-locked buffers of | ||||
|  * catalogs used in relcache, because a catcache search on any catalog may | ||||
|  * build that catalog's relcache entry.  We don't have an inventory of | ||||
|  * catalogs relcache uses, so just check buffers of most catalogs. | ||||
|  * | ||||
|  * It's better to minimize waits while holding an exclusive buffer lock, so it | ||||
|  * would be nice to broaden this check not to be catalog-specific.  However, | ||||
|  * bttextcmp() accesses pg_collation, and non-core opclasses might similarly | ||||
|  * read tables.  That is deadlock-free as long as there's no loop in the | ||||
|  * dependency graph: modifying table A may cause an opclass to read table B, | ||||
|  * but it must not cause a read of table A. | ||||
|  */ | ||||
| void | ||||
| AssertBufferLocksPermitCatalogRead(void) | ||||
| { | ||||
| 	ForEachLWLockHeldByMe(AssertNotCatalogBufferLock, NULL); | ||||
| } | ||||
|  | ||||
| static void | ||||
| AssertNotCatalogBufferLock(LWLock *lock, LWLockMode mode, | ||||
| 						   void *unused_context) | ||||
| { | ||||
| 	BufferDesc *bufHdr; | ||||
| 	BufferTag	tag; | ||||
| 	Oid			relid; | ||||
|  | ||||
| 	if (mode != LW_EXCLUSIVE) | ||||
| 		return; | ||||
|  | ||||
| 	if (!((BufferDescPadded *) lock > BufferDescriptors && | ||||
| 		  (BufferDescPadded *) lock < BufferDescriptors + NBuffers)) | ||||
| 		return;					/* not a buffer lock */ | ||||
|  | ||||
| 	bufHdr = (BufferDesc *) | ||||
| 		((char *) lock - offsetof(BufferDesc, content_lock)); | ||||
| 	tag = bufHdr->tag; | ||||
|  | ||||
| 	/* | ||||
| 	 * This relNumber==relid assumption holds until a catalog experiences | ||||
| 	 * VACUUM FULL or similar.  After a command like that, relNumber will be | ||||
| 	 * in the normal (non-catalog) range, and we lose the ability to detect | ||||
| 	 * hazardous access to that catalog.  Calling RelidByRelfilenumber() would | ||||
| 	 * close that gap, but RelidByRelfilenumber() might then deadlock with a | ||||
| 	 * held lock. | ||||
| 	 */ | ||||
| 	relid = tag.relNumber; | ||||
|  | ||||
| 	if (IsCatalogTextUniqueIndexOid(relid)) /* see comments at the callee */ | ||||
| 		return; | ||||
|  | ||||
| 	Assert(!IsCatalogRelationOid(relid)); | ||||
| 	/* Shared rels are always catalogs: detect even after VACUUM FULL. */ | ||||
| 	Assert(tag.spcOid != GLOBALTABLESPACE_OID); | ||||
| } | ||||
| #endif | ||||
|  | ||||
|  | ||||
| /* | ||||
|  * Helper routine to issue warnings when a buffer is unexpectedly pinned | ||||
|  */ | ||||
|   | ||||
| @@ -1961,6 +1961,21 @@ LWLockReleaseAll(void) | ||||
| } | ||||
|  | ||||
|  | ||||
| /* | ||||
|  * ForEachLWLockHeldByMe - run a callback for each held lock | ||||
|  * | ||||
|  * This is meant as debug support only. | ||||
|  */ | ||||
| void | ||||
| ForEachLWLockHeldByMe(void (*callback) (LWLock *, LWLockMode, void *), | ||||
| 					  void *context) | ||||
| { | ||||
| 	int			i; | ||||
|  | ||||
| 	for (i = 0; i < num_held_lwlocks; i++) | ||||
| 		callback(held_lwlocks[i].lock, held_lwlocks[i].mode, context); | ||||
| } | ||||
|  | ||||
| /* | ||||
|  * LWLockHeldByMe - test whether my process holds a lock in any mode | ||||
|  * | ||||
|   | ||||
| @@ -1196,6 +1196,8 @@ pg_newlocale_from_collation(Oid collid) | ||||
| 	if (!OidIsValid(collid)) | ||||
| 		elog(ERROR, "cache lookup failed for collation %u", collid); | ||||
|  | ||||
| 	AssertCouldGetRelation(); | ||||
|  | ||||
| 	if (last_collation_cache_oid == collid) | ||||
| 		return last_collation_cache_locale; | ||||
|  | ||||
|   | ||||
							
								
								
									
										51
									
								
								src/backend/utils/cache/catcache.c
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										51
									
								
								src/backend/utils/cache/catcache.c
									
									
									
									
										vendored
									
									
								
							| @@ -1054,12 +1054,41 @@ RehashCatCacheLists(CatCache *cp) | ||||
| 	cp->cc_lbucket = newbucket; | ||||
| } | ||||
|  | ||||
| /* | ||||
|  *		ConditionalCatalogCacheInitializeCache | ||||
|  * | ||||
|  * Call CatalogCacheInitializeCache() if not yet done. | ||||
|  */ | ||||
| pg_attribute_always_inline | ||||
| static void | ||||
| ConditionalCatalogCacheInitializeCache(CatCache *cache) | ||||
| { | ||||
| #ifdef USE_ASSERT_CHECKING | ||||
| 	/* | ||||
| 	 * TypeCacheRelCallback() runs outside transactions and relies on TYPEOID | ||||
| 	 * for hashing.  This isn't ideal.  Since lookup_type_cache() both | ||||
| 	 * registers the callback and searches TYPEOID, reaching trouble likely | ||||
| 	 * requires OOM at an unlucky moment. | ||||
| 	 * | ||||
| 	 * InvalidateAttoptCacheCallback() runs outside transactions and likewise | ||||
| 	 * relies on ATTNUM.  InitPostgres() initializes ATTNUM, so it's reliable. | ||||
| 	 */ | ||||
| 	if (!(cache->id == TYPEOID || cache->id == ATTNUM) || | ||||
| 		IsTransactionState()) | ||||
| 		AssertCouldGetRelation(); | ||||
| 	else | ||||
| 		Assert(cache->cc_tupdesc != NULL); | ||||
| #endif | ||||
|  | ||||
| 	if (unlikely(cache->cc_tupdesc == NULL)) | ||||
| 		CatalogCacheInitializeCache(cache); | ||||
| } | ||||
|  | ||||
| /* | ||||
|  *		CatalogCacheInitializeCache | ||||
|  * | ||||
|  * This function does final initialization of a catcache: obtain the tuple | ||||
|  * descriptor and set up the hash and equality function links.  We assume | ||||
|  * that the relcache entry can be opened at this point! | ||||
|  * descriptor and set up the hash and equality function links. | ||||
|  */ | ||||
| #ifdef CACHEDEBUG | ||||
| #define CatalogCacheInitializeCache_DEBUG1 \ | ||||
| @@ -1194,8 +1223,7 @@ CatalogCacheInitializeCache(CatCache *cache) | ||||
| void | ||||
| InitCatCachePhase2(CatCache *cache, bool touch_index) | ||||
| { | ||||
| 	if (cache->cc_tupdesc == NULL) | ||||
| 		CatalogCacheInitializeCache(cache); | ||||
| 	ConditionalCatalogCacheInitializeCache(cache); | ||||
|  | ||||
| 	if (touch_index && | ||||
| 		cache->id != AMOID && | ||||
| @@ -1374,16 +1402,12 @@ SearchCatCacheInternal(CatCache *cache, | ||||
| 	dlist_head *bucket; | ||||
| 	CatCTup    *ct; | ||||
|  | ||||
| 	/* Make sure we're in an xact, even if this ends up being a cache hit */ | ||||
| 	Assert(IsTransactionState()); | ||||
|  | ||||
| 	Assert(cache->cc_nkeys == nkeys); | ||||
|  | ||||
| 	/* | ||||
| 	 * one-time startup overhead for each cache | ||||
| 	 */ | ||||
| 	if (unlikely(cache->cc_tupdesc == NULL)) | ||||
| 		CatalogCacheInitializeCache(cache); | ||||
| 	ConditionalCatalogCacheInitializeCache(cache); | ||||
|  | ||||
| #ifdef CATCACHE_STATS | ||||
| 	cache->cc_searches++; | ||||
| @@ -1668,8 +1692,7 @@ GetCatCacheHashValue(CatCache *cache, | ||||
| 	/* | ||||
| 	 * one-time startup overhead for each cache | ||||
| 	 */ | ||||
| 	if (cache->cc_tupdesc == NULL) | ||||
| 		CatalogCacheInitializeCache(cache); | ||||
| 	ConditionalCatalogCacheInitializeCache(cache); | ||||
|  | ||||
| 	/* | ||||
| 	 * calculate the hash value | ||||
| @@ -1720,8 +1743,7 @@ SearchCatCacheList(CatCache *cache, | ||||
| 	/* | ||||
| 	 * one-time startup overhead for each cache | ||||
| 	 */ | ||||
| 	if (unlikely(cache->cc_tupdesc == NULL)) | ||||
| 		CatalogCacheInitializeCache(cache); | ||||
| 	ConditionalCatalogCacheInitializeCache(cache); | ||||
|  | ||||
| 	Assert(nkeys > 0 && nkeys < cache->cc_nkeys); | ||||
|  | ||||
| @@ -2390,8 +2412,7 @@ PrepareToInvalidateCacheTuple(Relation relation, | ||||
| 			continue; | ||||
|  | ||||
| 		/* Just in case cache hasn't finished initialization yet... */ | ||||
| 		if (ccp->cc_tupdesc == NULL) | ||||
| 			CatalogCacheInitializeCache(ccp); | ||||
| 		ConditionalCatalogCacheInitializeCache(ccp); | ||||
|  | ||||
| 		hashvalue = CatalogCacheComputeTupleHashValue(ccp, ccp->cc_nkeys, tuple); | ||||
| 		dbid = ccp->cc_relisshared ? (Oid) 0 : MyDatabaseId; | ||||
|   | ||||
							
								
								
									
										14
									
								
								src/backend/utils/cache/inval.c
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										14
									
								
								src/backend/utils/cache/inval.c
									
									
									
									
										vendored
									
									
								
							| @@ -683,7 +683,8 @@ PrepareInvalidationState(void) | ||||
| { | ||||
| 	TransInvalidationInfo *myInfo; | ||||
|  | ||||
| 	Assert(IsTransactionState()); | ||||
| 	/* PrepareToInvalidateCacheTuple() needs relcache */ | ||||
| 	AssertCouldGetRelation(); | ||||
| 	/* Can't queue transactional message while collecting inplace messages. */ | ||||
| 	Assert(inplaceInvalInfo == NULL); | ||||
|  | ||||
| @@ -752,7 +753,7 @@ PrepareInplaceInvalidationState(void) | ||||
| { | ||||
| 	InvalidationInfo *myInfo; | ||||
|  | ||||
| 	Assert(IsTransactionState()); | ||||
| 	AssertCouldGetRelation(); | ||||
| 	/* limit of one inplace update under assembly */ | ||||
| 	Assert(inplaceInvalInfo == NULL); | ||||
|  | ||||
| @@ -928,6 +929,12 @@ InvalidateSystemCaches(void) | ||||
| void | ||||
| AcceptInvalidationMessages(void) | ||||
| { | ||||
| #ifdef USE_ASSERT_CHECKING | ||||
| 	/* message handlers shall access catalogs only during transactions */ | ||||
| 	if (IsTransactionState()) | ||||
| 		AssertCouldGetRelation(); | ||||
| #endif | ||||
|  | ||||
| 	ReceiveSharedInvalidMessages(LocalExecuteInvalidationMessage, | ||||
| 								 InvalidateSystemCaches); | ||||
|  | ||||
| @@ -1436,6 +1443,9 @@ CacheInvalidateHeapTupleCommon(Relation relation, | ||||
| 	Oid			databaseId; | ||||
| 	Oid			relationId; | ||||
|  | ||||
| 	/* PrepareToInvalidateCacheTuple() needs relcache */ | ||||
| 	AssertCouldGetRelation(); | ||||
|  | ||||
| 	/* Do nothing during bootstrap */ | ||||
| 	if (IsBootstrapProcessingMode()) | ||||
| 		return; | ||||
|   | ||||
							
								
								
									
										26
									
								
								src/backend/utils/cache/relcache.c
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										26
									
								
								src/backend/utils/cache/relcache.c
									
									
									
									
										vendored
									
									
								
							| @@ -2056,6 +2056,23 @@ formrdesc(const char *relationName, Oid relationReltype, | ||||
| 	relation->rd_isvalid = true; | ||||
| } | ||||
|  | ||||
| #ifdef USE_ASSERT_CHECKING | ||||
| /* | ||||
|  *		AssertCouldGetRelation | ||||
|  * | ||||
|  *		Check safety of calling RelationIdGetRelation(). | ||||
|  * | ||||
|  *		In code that reads catalogs in the event of a cache miss, call this | ||||
|  *		before checking the cache. | ||||
|  */ | ||||
| void | ||||
| AssertCouldGetRelation(void) | ||||
| { | ||||
| 	Assert(IsTransactionState()); | ||||
| 	AssertBufferLocksPermitCatalogRead(); | ||||
| } | ||||
| #endif | ||||
|  | ||||
|  | ||||
| /* ---------------------------------------------------------------- | ||||
|  *				 Relation Descriptor Lookup Interface | ||||
| @@ -2083,8 +2100,7 @@ RelationIdGetRelation(Oid relationId) | ||||
| { | ||||
| 	Relation	rd; | ||||
|  | ||||
| 	/* Make sure we're in an xact, even if this ends up being a cache hit */ | ||||
| 	Assert(IsTransactionState()); | ||||
| 	AssertCouldGetRelation(); | ||||
|  | ||||
| 	/* | ||||
| 	 * first try to find reldesc in the cache | ||||
| @@ -2373,8 +2389,7 @@ RelationReloadNailed(Relation relation) | ||||
| 	Assert(relation->rd_isnailed); | ||||
| 	/* nailed indexes are handled by RelationReloadIndexInfo() */ | ||||
| 	Assert(relation->rd_rel->relkind == RELKIND_RELATION); | ||||
| 	/* can only reread catalog contents in a transaction */ | ||||
| 	Assert(IsTransactionState()); | ||||
| 	AssertCouldGetRelation(); | ||||
|  | ||||
| 	/* | ||||
| 	 * Redo RelationInitPhysicalAddr in case it is a mapped relation whose | ||||
| @@ -2570,8 +2585,7 @@ static void | ||||
| RelationRebuildRelation(Relation relation) | ||||
| { | ||||
| 	Assert(!RelationHasReferenceCountZero(relation)); | ||||
| 	/* rebuilding requires access to the catalogs */ | ||||
| 	Assert(IsTransactionState()); | ||||
| 	AssertCouldGetRelation(); | ||||
| 	/* there is no reason to ever rebuild a dropped relation */ | ||||
| 	Assert(relation->rd_droppedSubid == InvalidSubTransactionId); | ||||
|  | ||||
|   | ||||
| @@ -310,7 +310,7 @@ InitializeClientEncoding(void) | ||||
| 	{ | ||||
| 		Oid			utf8_to_server_proc; | ||||
|  | ||||
| 		Assert(IsTransactionState()); | ||||
| 		AssertCouldGetRelation(); | ||||
| 		utf8_to_server_proc = | ||||
| 			FindDefaultConversionProc(PG_UTF8, | ||||
| 									  current_server_encoding); | ||||
|   | ||||
| @@ -27,6 +27,7 @@ extern bool IsSystemClass(Oid relid, Form_pg_class reltuple); | ||||
| extern bool IsToastClass(Form_pg_class reltuple); | ||||
|  | ||||
| extern bool IsCatalogRelationOid(Oid relid); | ||||
| extern bool IsCatalogTextUniqueIndexOid(Oid relid); | ||||
| extern bool IsInplaceUpdateOid(Oid relid); | ||||
|  | ||||
| extern bool IsCatalogNamespace(Oid namespaceId); | ||||
|   | ||||
| @@ -258,6 +258,9 @@ extern Buffer ExtendBufferedRelTo(BufferManagerRelation bmr, | ||||
|  | ||||
| extern void InitBufferManagerAccess(void); | ||||
| extern void AtEOXact_Buffers(bool isCommit); | ||||
| #ifdef USE_ASSERT_CHECKING | ||||
| extern void AssertBufferLocksPermitCatalogRead(void); | ||||
| #endif | ||||
| extern char *DebugPrintBufferRefcount(Buffer buffer); | ||||
| extern void CheckPointBuffers(int flags); | ||||
| extern BlockNumber BufferGetBlockNumber(Buffer buffer); | ||||
|   | ||||
| @@ -131,6 +131,8 @@ extern void LWLockReleaseClearVar(LWLock *lock, pg_atomic_uint64 *valptr, uint64 | ||||
| extern void LWLockReleaseAll(void); | ||||
| extern void LWLockDisown(LWLock *lock); | ||||
| extern void LWLockReleaseDisowned(LWLock *lock, LWLockMode mode); | ||||
| extern void ForEachLWLockHeldByMe(void (*callback) (LWLock *, LWLockMode, void *), | ||||
| 								  void *context); | ||||
| extern bool LWLockHeldByMe(LWLock *lock); | ||||
| extern bool LWLockAnyHeldByMe(LWLock *lock, int nlocks, size_t stride); | ||||
| extern bool LWLockHeldByMeInMode(LWLock *lock, LWLockMode mode); | ||||
|   | ||||
| @@ -37,6 +37,14 @@ typedef Relation *RelationPtr; | ||||
| /* | ||||
|  * Routines to open (lookup) and close a relcache entry | ||||
|  */ | ||||
| #ifdef USE_ASSERT_CHECKING | ||||
| extern void AssertCouldGetRelation(void); | ||||
| #else | ||||
| static inline void | ||||
| AssertCouldGetRelation(void) | ||||
| { | ||||
| } | ||||
| #endif | ||||
| extern Relation RelationIdGetRelation(Oid relationId); | ||||
| extern void RelationClose(Relation relation); | ||||
|  | ||||
|   | ||||
| @@ -11,6 +11,10 @@ | ||||
| -- that is OID or REGPROC fields that are not zero and do not match some | ||||
| -- row in the linked-to table.  However, if we want to enforce that a link | ||||
| -- field can't be 0, we have to check it here. | ||||
| -- directory paths and dlsuffix are passed to us in environment variables | ||||
| \getenv libdir PG_LIBDIR | ||||
| \getenv dlsuffix PG_DLSUFFIX | ||||
| \set regresslib :libdir '/regress' :dlsuffix | ||||
| -- **************** pg_type **************** | ||||
| -- Look for illegal values in pg_type fields. | ||||
| SELECT t1.oid, t1.typname | ||||
| @@ -587,6 +591,21 @@ WHERE a1.atttypid = t1.oid AND | ||||
| ----------+---------+-----+--------- | ||||
| (0 rows) | ||||
|  | ||||
| -- Look for IsCatalogTextUniqueIndexOid() omissions. | ||||
| CREATE FUNCTION is_catalog_text_unique_index_oid(oid) RETURNS bool | ||||
|     AS :'regresslib', 'is_catalog_text_unique_index_oid' | ||||
|     LANGUAGE C STRICT; | ||||
| SELECT indexrelid::regclass | ||||
| FROM pg_index | ||||
| WHERE (is_catalog_text_unique_index_oid(indexrelid) <> | ||||
|        (indisunique AND | ||||
|         indexrelid < 16384 AND | ||||
|         EXISTS (SELECT 1 FROM pg_attribute | ||||
|                 WHERE attrelid = indexrelid AND atttypid = 'text'::regtype))); | ||||
|  indexrelid  | ||||
| ------------ | ||||
| (0 rows) | ||||
|  | ||||
| -- **************** pg_range **************** | ||||
| -- Look for illegal values in pg_range fields. | ||||
| SELECT r.rngtypid, r.rngsubtype | ||||
|   | ||||
| @@ -21,6 +21,7 @@ | ||||
|  | ||||
| #include "access/detoast.h" | ||||
| #include "access/htup_details.h" | ||||
| #include "catalog/catalog.h" | ||||
| #include "catalog/namespace.h" | ||||
| #include "catalog/pg_operator.h" | ||||
| #include "catalog/pg_type.h" | ||||
| @@ -722,6 +723,13 @@ test_fdw_handler(PG_FUNCTION_ARGS) | ||||
| 	PG_RETURN_NULL(); | ||||
| } | ||||
|  | ||||
| PG_FUNCTION_INFO_V1(is_catalog_text_unique_index_oid); | ||||
| Datum | ||||
| is_catalog_text_unique_index_oid(PG_FUNCTION_ARGS) | ||||
| { | ||||
| 	return IsCatalogTextUniqueIndexOid(PG_GETARG_OID(0)); | ||||
| } | ||||
|  | ||||
| PG_FUNCTION_INFO_V1(test_support_func); | ||||
| Datum | ||||
| test_support_func(PG_FUNCTION_ARGS) | ||||
|   | ||||
| @@ -12,6 +12,12 @@ | ||||
| -- row in the linked-to table.  However, if we want to enforce that a link | ||||
| -- field can't be 0, we have to check it here. | ||||
|  | ||||
| -- directory paths and dlsuffix are passed to us in environment variables | ||||
| \getenv libdir PG_LIBDIR | ||||
| \getenv dlsuffix PG_DLSUFFIX | ||||
|  | ||||
| \set regresslib :libdir '/regress' :dlsuffix | ||||
|  | ||||
| -- **************** pg_type **************** | ||||
|  | ||||
| -- Look for illegal values in pg_type fields. | ||||
| @@ -425,6 +431,20 @@ WHERE a1.atttypid = t1.oid AND | ||||
|      a1.attbyval != t1.typbyval OR | ||||
|      (a1.attstorage != t1.typstorage AND a1.attstorage != 'p')); | ||||
|  | ||||
| -- Look for IsCatalogTextUniqueIndexOid() omissions. | ||||
|  | ||||
| CREATE FUNCTION is_catalog_text_unique_index_oid(oid) RETURNS bool | ||||
|     AS :'regresslib', 'is_catalog_text_unique_index_oid' | ||||
|     LANGUAGE C STRICT; | ||||
|  | ||||
| SELECT indexrelid::regclass | ||||
| FROM pg_index | ||||
| WHERE (is_catalog_text_unique_index_oid(indexrelid) <> | ||||
|        (indisunique AND | ||||
|         indexrelid < 16384 AND | ||||
|         EXISTS (SELECT 1 FROM pg_attribute | ||||
|                 WHERE attrelid = indexrelid AND atttypid = 'text'::regtype))); | ||||
|  | ||||
| -- **************** pg_range **************** | ||||
|  | ||||
| -- Look for illegal values in pg_range fields. | ||||
|   | ||||
		Reference in New Issue
	
	Block a user