PostgreSQL Source Code  git master
predicate.c
Go to the documentation of this file.
1 /*-------------------------------------------------------------------------
2  *
3  * predicate.c
4  * POSTGRES predicate locking
5  * to support full serializable transaction isolation
6  *
7  *
8  * The approach taken is to implement Serializable Snapshot Isolation (SSI)
9  * as initially described in this paper:
10  *
11  * Michael J. Cahill, Uwe Röhm, and Alan D. Fekete. 2008.
12  * Serializable isolation for snapshot databases.
13  * In SIGMOD '08: Proceedings of the 2008 ACM SIGMOD
14  * international conference on Management of data,
15  * pages 729-738, New York, NY, USA. ACM.
16  * http://doi.acm.org/10.1145/1376616.1376690
17  *
18  * and further elaborated in Cahill's doctoral thesis:
19  *
20  * Michael James Cahill. 2009.
21  * Serializable Isolation for Snapshot Databases.
22  * Sydney Digital Theses.
23  * University of Sydney, School of Information Technologies.
24  * http://hdl.handle.net/2123/5353
25  *
26  *
27  * Predicate locks for Serializable Snapshot Isolation (SSI) are SIREAD
28  * locks, which are so different from normal locks that a distinct set of
29  * structures is required to handle them. They are needed to detect
30  * rw-conflicts when the read happens before the write. (When the write
31  * occurs first, the reading transaction can check for a conflict by
32  * examining the MVCC data.)
33  *
34  * (1) Besides tuples actually read, they must cover ranges of tuples
35  * which would have been read based on the predicate. This will
36  * require modelling the predicates through locks against database
37  * objects such as pages, index ranges, or entire tables.
38  *
39  * (2) They must be kept in RAM for quick access. Because of this, it
40  * isn't possible to always maintain tuple-level granularity -- when
41  * the space allocated to store these approaches exhaustion, a
42  * request for a lock may need to scan for situations where a single
43  * transaction holds many fine-grained locks which can be coalesced
44  * into a single coarser-grained lock.
45  *
46  * (3) They never block anything; they are more like flags than locks
47  * in that regard; although they refer to database objects and are
48  * used to identify rw-conflicts with normal write locks.
49  *
50  * (4) While they are associated with a transaction, they must survive
51  * a successful COMMIT of that transaction, and remain until all
52  * overlapping transactions complete. This even means that they
53  * must survive termination of the transaction's process. If a
54  * top level transaction is rolled back, however, it is immediately
55  * flagged so that it can be ignored, and its SIREAD locks can be
56  * released any time after that.
57  *
58  * (5) The only transactions which create SIREAD locks or check for
59  * conflicts with them are serializable transactions.
60  *
61  * (6) When a write lock for a top level transaction is found to cover
62  * an existing SIREAD lock for the same transaction, the SIREAD lock
63  * can be deleted.
64  *
65  * (7) A write from a serializable transaction must ensure that an xact
66  * record exists for the transaction, with the same lifespan (until
67  * all concurrent transaction complete or the transaction is rolled
68  * back) so that rw-dependencies to that transaction can be
69  * detected.
70  *
71  * We use an optimization for read-only transactions. Under certain
72  * circumstances, a read-only transaction's snapshot can be shown to
73  * never have conflicts with other transactions. This is referred to
74  * as a "safe" snapshot (and one known not to be is "unsafe").
75  * However, it can't be determined whether a snapshot is safe until
76  * all concurrent read/write transactions complete.
77  *
78  * Once a read-only transaction is known to have a safe snapshot, it
79  * can release its predicate locks and exempt itself from further
80  * predicate lock tracking. READ ONLY DEFERRABLE transactions run only
81  * on safe snapshots, waiting as necessary for one to be available.
82  *
83  *
84  * Lightweight locks to manage access to the predicate locking shared
85  * memory objects must be taken in this order, and should be released in
86  * reverse order:
87  *
88  * SerializableFinishedListLock
89  * - Protects the list of transactions which have completed but which
90  * may yet matter because they overlap still-active transactions.
91  *
92  * SerializablePredicateListLock
93  * - Protects the linked list of locks held by a transaction. Note
94  * that the locks themselves are also covered by the partition
95  * locks of their respective lock targets; this lock only affects
96  * the linked list connecting the locks related to a transaction.
97  * - All transactions share this single lock (with no partitioning).
98  * - There is never a need for a process other than the one running
99  * an active transaction to walk the list of locks held by that
100  * transaction, except parallel query workers sharing the leader's
101  * transaction. In the parallel case, an extra per-sxact lock is
102  * taken; see below.
103  * - It is relatively infrequent that another process needs to
104  * modify the list for a transaction, but it does happen for such
105  * things as index page splits for pages with predicate locks and
106  * freeing of predicate locked pages by a vacuum process. When
107  * removing a lock in such cases, the lock itself contains the
108  * pointers needed to remove it from the list. When adding a
109  * lock in such cases, the lock can be added using the anchor in
110  * the transaction structure. Neither requires walking the list.
111  * - Cleaning up the list for a terminated transaction is sometimes
112  * not done on a retail basis, in which case no lock is required.
113  * - Due to the above, a process accessing its active transaction's
114  * list always uses a shared lock, regardless of whether it is
115  * walking or maintaining the list. This improves concurrency
116  * for the common access patterns.
117  * - A process which needs to alter the list of a transaction other
118  * than its own active transaction must acquire an exclusive
119  * lock.
120  *
121  * SERIALIZABLEXACT's member 'perXactPredicateListLock'
122  * - Protects the linked list of predicate locks held by a transaction.
123  * Only needed for parallel mode, where multiple backends share the
124  * same SERIALIZABLEXACT object. Not needed if
125  * SerializablePredicateListLock is held exclusively.
126  *
127  * PredicateLockHashPartitionLock(hashcode)
128  * - The same lock protects a target, all locks on that target, and
129  * the linked list of locks on the target.
130  * - When more than one is needed, acquire in ascending address order.
131  * - When all are needed (rare), acquire in ascending index order with
132  * PredicateLockHashPartitionLockByIndex(index).
133  *
134  * SerializableXactHashLock
135  * - Protects both PredXact and SerializableXidHash.
136  *
137  * SerialControlLock
138  * - Protects SerialControlData members
139  *
140  * SLRU per-bank locks
141  * - Protects SerialSlruCtl
142  *
143  * Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
144  * Portions Copyright (c) 1994, Regents of the University of California
145  *
146  *
147  * IDENTIFICATION
148  * src/backend/storage/lmgr/predicate.c
149  *
150  *-------------------------------------------------------------------------
151  */
152 /*
153  * INTERFACE ROUTINES
154  *
155  * housekeeping for setting up shared memory predicate lock structures
156  * InitPredicateLocks(void)
157  * PredicateLockShmemSize(void)
158  *
159  * predicate lock reporting
160  * GetPredicateLockStatusData(void)
161  * PageIsPredicateLocked(Relation relation, BlockNumber blkno)
162  *
163  * predicate lock maintenance
164  * GetSerializableTransactionSnapshot(Snapshot snapshot)
165  * SetSerializableTransactionSnapshot(Snapshot snapshot,
166  * VirtualTransactionId *sourcevxid)
167  * RegisterPredicateLockingXid(void)
168  * PredicateLockRelation(Relation relation, Snapshot snapshot)
169  * PredicateLockPage(Relation relation, BlockNumber blkno,
170  * Snapshot snapshot)
171  * PredicateLockTID(Relation relation, ItemPointer tid, Snapshot snapshot,
172  * TransactionId tuple_xid)
173  * PredicateLockPageSplit(Relation relation, BlockNumber oldblkno,
174  * BlockNumber newblkno)
175  * PredicateLockPageCombine(Relation relation, BlockNumber oldblkno,
176  * BlockNumber newblkno)
177  * TransferPredicateLocksToHeapRelation(Relation relation)
178  * ReleasePredicateLocks(bool isCommit, bool isReadOnlySafe)
179  *
180  * conflict detection (may also trigger rollback)
181  * CheckForSerializableConflictOut(Relation relation, TransactionId xid,
182  * Snapshot snapshot)
183  * CheckForSerializableConflictIn(Relation relation, ItemPointer tid,
184  * BlockNumber blkno)
185  * CheckTableForSerializableConflictIn(Relation relation)
186  *
187  * final rollback checking
188  * PreCommit_CheckForSerializationFailure(void)
189  *
190  * two-phase commit support
191  * AtPrepare_PredicateLocks(void);
192  * PostPrepare_PredicateLocks(TransactionId xid);
193  * PredicateLockTwoPhaseFinish(TransactionId xid, bool isCommit);
194  * predicatelock_twophase_recover(TransactionId xid, uint16 info,
195  * void *recdata, uint32 len);
196  */
197 
198 #include "postgres.h"
199 
200 #include "access/parallel.h"
201 #include "access/slru.h"
202 #include "access/transam.h"
203 #include "access/twophase.h"
204 #include "access/twophase_rmgr.h"
205 #include "access/xact.h"
206 #include "access/xlog.h"
207 #include "miscadmin.h"
208 #include "pgstat.h"
209 #include "port/pg_lfind.h"
210 #include "storage/predicate.h"
212 #include "storage/proc.h"
213 #include "storage/procarray.h"
214 #include "utils/guc_hooks.h"
215 #include "utils/rel.h"
216 #include "utils/snapmgr.h"
217 
218 /* Uncomment the next line to test the graceful degradation code. */
219 /* #define TEST_SUMMARIZE_SERIAL */
220 
221 /*
222  * Test the most selective fields first, for performance.
223  *
224  * a is covered by b if all of the following hold:
225  * 1) a.database = b.database
226  * 2) a.relation = b.relation
227  * 3) b.offset is invalid (b is page-granularity or higher)
228  * 4) either of the following:
229  * 4a) a.offset is valid (a is tuple-granularity) and a.page = b.page
230  * or 4b) a.offset is invalid and b.page is invalid (a is
231  * page-granularity and b is relation-granularity
232  */
233 #define TargetTagIsCoveredBy(covered_target, covering_target) \
234  ((GET_PREDICATELOCKTARGETTAG_RELATION(covered_target) == /* (2) */ \
235  GET_PREDICATELOCKTARGETTAG_RELATION(covering_target)) \
236  && (GET_PREDICATELOCKTARGETTAG_OFFSET(covering_target) == \
237  InvalidOffsetNumber) /* (3) */ \
238  && (((GET_PREDICATELOCKTARGETTAG_OFFSET(covered_target) != \
239  InvalidOffsetNumber) /* (4a) */ \
240  && (GET_PREDICATELOCKTARGETTAG_PAGE(covering_target) == \
241  GET_PREDICATELOCKTARGETTAG_PAGE(covered_target))) \
242  || ((GET_PREDICATELOCKTARGETTAG_PAGE(covering_target) == \
243  InvalidBlockNumber) /* (4b) */ \
244  && (GET_PREDICATELOCKTARGETTAG_PAGE(covered_target) \
245  != InvalidBlockNumber))) \
246  && (GET_PREDICATELOCKTARGETTAG_DB(covered_target) == /* (1) */ \
247  GET_PREDICATELOCKTARGETTAG_DB(covering_target)))
248 
249 /*
250  * The predicate locking target and lock shared hash tables are partitioned to
251  * reduce contention. To determine which partition a given target belongs to,
252  * compute the tag's hash code with PredicateLockTargetTagHashCode(), then
253  * apply one of these macros.
254  * NB: NUM_PREDICATELOCK_PARTITIONS must be a power of 2!
255  */
256 #define PredicateLockHashPartition(hashcode) \
257  ((hashcode) % NUM_PREDICATELOCK_PARTITIONS)
258 #define PredicateLockHashPartitionLock(hashcode) \
259  (&MainLWLockArray[PREDICATELOCK_MANAGER_LWLOCK_OFFSET + \
260  PredicateLockHashPartition(hashcode)].lock)
261 #define PredicateLockHashPartitionLockByIndex(i) \
262  (&MainLWLockArray[PREDICATELOCK_MANAGER_LWLOCK_OFFSET + (i)].lock)
263 
264 #define NPREDICATELOCKTARGETENTS() \
265  mul_size(max_predicate_locks_per_xact, add_size(MaxBackends, max_prepared_xacts))
266 
267 #define SxactIsOnFinishedList(sxact) (!dlist_node_is_detached(&(sxact)->finishedLink))
268 
269 /*
270  * Note that a sxact is marked "prepared" once it has passed
271  * PreCommit_CheckForSerializationFailure, even if it isn't using
272  * 2PC. This is the point at which it can no longer be aborted.
273  *
274  * The PREPARED flag remains set after commit, so SxactIsCommitted
275  * implies SxactIsPrepared.
276  */
277 #define SxactIsCommitted(sxact) (((sxact)->flags & SXACT_FLAG_COMMITTED) != 0)
278 #define SxactIsPrepared(sxact) (((sxact)->flags & SXACT_FLAG_PREPARED) != 0)
279 #define SxactIsRolledBack(sxact) (((sxact)->flags & SXACT_FLAG_ROLLED_BACK) != 0)
280 #define SxactIsDoomed(sxact) (((sxact)->flags & SXACT_FLAG_DOOMED) != 0)
281 #define SxactIsReadOnly(sxact) (((sxact)->flags & SXACT_FLAG_READ_ONLY) != 0)
282 #define SxactHasSummaryConflictIn(sxact) (((sxact)->flags & SXACT_FLAG_SUMMARY_CONFLICT_IN) != 0)
283 #define SxactHasSummaryConflictOut(sxact) (((sxact)->flags & SXACT_FLAG_SUMMARY_CONFLICT_OUT) != 0)
284 /*
285  * The following macro actually means that the specified transaction has a
286  * conflict out *to a transaction which committed ahead of it*. It's hard
287  * to get that into a name of a reasonable length.
288  */
289 #define SxactHasConflictOut(sxact) (((sxact)->flags & SXACT_FLAG_CONFLICT_OUT) != 0)
290 #define SxactIsDeferrableWaiting(sxact) (((sxact)->flags & SXACT_FLAG_DEFERRABLE_WAITING) != 0)
291 #define SxactIsROSafe(sxact) (((sxact)->flags & SXACT_FLAG_RO_SAFE) != 0)
292 #define SxactIsROUnsafe(sxact) (((sxact)->flags & SXACT_FLAG_RO_UNSAFE) != 0)
293 #define SxactIsPartiallyReleased(sxact) (((sxact)->flags & SXACT_FLAG_PARTIALLY_RELEASED) != 0)
294 
295 /*
296  * Compute the hash code associated with a PREDICATELOCKTARGETTAG.
297  *
298  * To avoid unnecessary recomputations of the hash code, we try to do this
299  * just once per function, and then pass it around as needed. Aside from
300  * passing the hashcode to hash_search_with_hash_value(), we can extract
301  * the lock partition number from the hashcode.
302  */
303 #define PredicateLockTargetTagHashCode(predicatelocktargettag) \
304  get_hash_value(PredicateLockTargetHash, predicatelocktargettag)
305 
306 /*
307  * Given a predicate lock tag, and the hash for its target,
308  * compute the lock hash.
309  *
310  * To make the hash code also depend on the transaction, we xor the sxid
311  * struct's address into the hash code, left-shifted so that the
312  * partition-number bits don't change. Since this is only a hash, we
313  * don't care if we lose high-order bits of the address; use an
314  * intermediate variable to suppress cast-pointer-to-int warnings.
315  */
316 #define PredicateLockHashCodeFromTargetHashCode(predicatelocktag, targethash) \
317  ((targethash) ^ ((uint32) PointerGetDatum((predicatelocktag)->myXact)) \
318  << LOG2_NUM_PREDICATELOCK_PARTITIONS)
319 
320 
321 /*
322  * The SLRU buffer area through which we access the old xids.
323  */
325 
326 #define SerialSlruCtl (&SerialSlruCtlData)
327 
328 #define SERIAL_PAGESIZE BLCKSZ
329 #define SERIAL_ENTRYSIZE sizeof(SerCommitSeqNo)
330 #define SERIAL_ENTRIESPERPAGE (SERIAL_PAGESIZE / SERIAL_ENTRYSIZE)
331 
332 /*
333  * Set maximum pages based on the number needed to track all transactions.
334  */
335 #define SERIAL_MAX_PAGE (MaxTransactionId / SERIAL_ENTRIESPERPAGE)
336 
337 #define SerialNextPage(page) (((page) >= SERIAL_MAX_PAGE) ? 0 : (page) + 1)
338 
339 #define SerialValue(slotno, xid) (*((SerCommitSeqNo *) \
340  (SerialSlruCtl->shared->page_buffer[slotno] + \
341  ((((uint32) (xid)) % SERIAL_ENTRIESPERPAGE) * SERIAL_ENTRYSIZE))))
342 
343 #define SerialPage(xid) (((uint32) (xid)) / SERIAL_ENTRIESPERPAGE)
344 
345 typedef struct SerialControlData
346 {
347  int headPage; /* newest initialized page */
348  TransactionId headXid; /* newest valid Xid in the SLRU */
349  TransactionId tailXid; /* oldest xmin we might be interested in */
351 
353 
355 
356 /*
357  * When the oldest committed transaction on the "finished" list is moved to
358  * SLRU, its predicate locks will be moved to this "dummy" transaction,
359  * collapsing duplicate targets. When a duplicate is found, the later
360  * commitSeqNo is used.
361  */
363 
364 
365 /*
366  * These configuration variables are used to set the predicate lock table size
367  * and to control promotion of predicate locks to coarser granularity in an
368  * attempt to degrade performance (mostly as false positive serialization
369  * failure) gracefully in the face of memory pressure.
370  */
371 int max_predicate_locks_per_xact; /* in guc_tables.c */
372 int max_predicate_locks_per_relation; /* in guc_tables.c */
373 int max_predicate_locks_per_page; /* in guc_tables.c */
374 
375 /*
376  * This provides a list of objects in order to track transactions
377  * participating in predicate locking. Entries in the list are fixed size,
378  * and reside in shared memory. The memory address of an entry must remain
379  * fixed during its lifetime. The list will be protected from concurrent
380  * update externally; no provision is made in this code to manage that. The
381  * number of entries in the list, and the size allowed for each entry is
382  * fixed upon creation.
383  */
385 
386 /*
387  * This provides a pool of RWConflict data elements to use in conflict lists
388  * between transactions.
389  */
391 
392 /*
393  * The predicate locking hash tables are in shared memory.
394  * Each backend keeps pointers to them.
395  */
400 
401 /*
402  * Tag for a dummy entry in PredicateLockTargetHash. By temporarily removing
403  * this entry, you can ensure that there's enough scratch space available for
404  * inserting one entry in the hash table. This is an otherwise-invalid tag.
405  */
406 static const PREDICATELOCKTARGETTAG ScratchTargetTag = {0, 0, 0, 0};
409 
410 /*
411  * The local hash table used to determine when to combine multiple fine-
412  * grained locks into a single courser-grained lock.
413  */
415 
416 /*
417  * Keep a pointer to the currently-running serializable transaction (if any)
418  * for quick reference. Also, remember if we have written anything that could
419  * cause a rw-conflict.
420  */
422 static bool MyXactDidWrite = false;
423 
424 /*
425  * The SXACT_FLAG_RO_UNSAFE optimization might lead us to release
426  * MySerializableXact early. If that happens in a parallel query, the leader
427  * needs to defer the destruction of the SERIALIZABLEXACT until end of
428  * transaction, because the workers still have a reference to it. In that
429  * case, the leader stores it here.
430  */
432 
433 /* local functions */
434 
435 static SERIALIZABLEXACT *CreatePredXact(void);
436 static void ReleasePredXact(SERIALIZABLEXACT *sxact);
437 
438 static bool RWConflictExists(const SERIALIZABLEXACT *reader, const SERIALIZABLEXACT *writer);
439 static void SetRWConflict(SERIALIZABLEXACT *reader, SERIALIZABLEXACT *writer);
440 static void SetPossibleUnsafeConflict(SERIALIZABLEXACT *roXact, SERIALIZABLEXACT *activeXact);
441 static void ReleaseRWConflict(RWConflict conflict);
442 static void FlagSxactUnsafe(SERIALIZABLEXACT *sxact);
443 
444 static bool SerialPagePrecedesLogically(int64 page1, int64 page2);
445 static void SerialInit(void);
446 static void SerialAdd(TransactionId xid, SerCommitSeqNo minConflictCommitSeqNo);
448 static void SerialSetActiveSerXmin(TransactionId xid);
449 
450 static uint32 predicatelock_hash(const void *key, Size keysize);
451 static void SummarizeOldestCommittedSxact(void);
452 static Snapshot GetSafeSnapshot(Snapshot origSnapshot);
454  VirtualTransactionId *sourcevxid,
455  int sourcepid);
456 static bool PredicateLockExists(const PREDICATELOCKTARGETTAG *targettag);
458  PREDICATELOCKTARGETTAG *parent);
459 static bool CoarserLockCovers(const PREDICATELOCKTARGETTAG *newtargettag);
460 static void RemoveScratchTarget(bool lockheld);
461 static void RestoreScratchTarget(bool lockheld);
463  uint32 targettaghash);
464 static void DeleteChildTargetLocks(const PREDICATELOCKTARGETTAG *newtargettag);
465 static int MaxPredicateChildLocks(const PREDICATELOCKTARGETTAG *tag);
467 static void DecrementParentLocks(const PREDICATELOCKTARGETTAG *targettag);
468 static void CreatePredicateLock(const PREDICATELOCKTARGETTAG *targettag,
469  uint32 targettaghash,
470  SERIALIZABLEXACT *sxact);
471 static void DeleteLockTarget(PREDICATELOCKTARGET *target, uint32 targettaghash);
473  PREDICATELOCKTARGETTAG newtargettag,
474  bool removeOld);
475 static void PredicateLockAcquire(const PREDICATELOCKTARGETTAG *targettag);
476 static void DropAllPredicateLocksFromTable(Relation relation,
477  bool transfer);
478 static void SetNewSxactGlobalXmin(void);
479 static void ClearOldPredicateLocks(void);
480 static void ReleaseOneSerializableXact(SERIALIZABLEXACT *sxact, bool partial,
481  bool summarize);
482 static bool XidIsConcurrent(TransactionId xid);
483 static void CheckTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag);
484 static void FlagRWConflict(SERIALIZABLEXACT *reader, SERIALIZABLEXACT *writer);
486  SERIALIZABLEXACT *writer);
487 static void CreateLocalPredicateLockHash(void);
488 static void ReleasePredicateLocksLocal(void);
489 
490 
491 /*------------------------------------------------------------------------*/
492 
493 /*
494  * Does this relation participate in predicate locking? Temporary and system
495  * relations are exempt.
496  */
497 static inline bool
499 {
500  return !(relation->rd_id < FirstUnpinnedObjectId ||
501  RelationUsesLocalBuffers(relation));
502 }
503 
504 /*
505  * When a public interface method is called for a read, this is the test to
506  * see if we should do a quick return.
507  *
508  * Note: this function has side-effects! If this transaction has been flagged
509  * as RO-safe since the last call, we release all predicate locks and reset
510  * MySerializableXact. That makes subsequent calls to return quickly.
511  *
512  * This is marked as 'inline' to eliminate the function call overhead in the
513  * common case that serialization is not needed.
514  */
515 static inline bool
517 {
518  /* Nothing to do if this is not a serializable transaction */
520  return false;
521 
522  /*
523  * Don't acquire locks or conflict when scanning with a special snapshot.
524  * This excludes things like CLUSTER and REINDEX. They use the wholesale
525  * functions TransferPredicateLocksToHeapRelation() and
526  * CheckTableForSerializableConflictIn() to participate in serialization,
527  * but the scans involved don't need serialization.
528  */
529  if (!IsMVCCSnapshot(snapshot))
530  return false;
531 
532  /*
533  * Check if we have just become "RO-safe". If we have, immediately release
534  * all locks as they're not needed anymore. This also resets
535  * MySerializableXact, so that subsequent calls to this function can exit
536  * quickly.
537  *
538  * A transaction is flagged as RO_SAFE if all concurrent R/W transactions
539  * commit without having conflicts out to an earlier snapshot, thus
540  * ensuring that no conflicts are possible for this transaction.
541  */
543  {
544  ReleasePredicateLocks(false, true);
545  return false;
546  }
547 
548  /* Check if the relation doesn't participate in predicate locking */
549  if (!PredicateLockingNeededForRelation(relation))
550  return false;
551 
552  return true; /* no excuse to skip predicate locking */
553 }
554 
555 /*
556  * Like SerializationNeededForRead(), but called on writes.
557  * The logic is the same, but there is no snapshot and we can't be RO-safe.
558  */
559 static inline bool
561 {
562  /* Nothing to do if this is not a serializable transaction */
564  return false;
565 
566  /* Check if the relation doesn't participate in predicate locking */
567  if (!PredicateLockingNeededForRelation(relation))
568  return false;
569 
570  return true; /* no excuse to skip predicate locking */
571 }
572 
573 
574 /*------------------------------------------------------------------------*/
575 
576 /*
577  * These functions are a simple implementation of a list for this specific
578  * type of struct. If there is ever a generalized shared memory list, we
579  * should probably switch to that.
580  */
581 static SERIALIZABLEXACT *
583 {
584  SERIALIZABLEXACT *sxact;
585 
587  return NULL;
588 
589  sxact = dlist_container(SERIALIZABLEXACT, xactLink,
592  return sxact;
593 }
594 
595 static void
597 {
598  Assert(ShmemAddrIsValid(sxact));
599 
600  dlist_delete(&sxact->xactLink);
602 }
603 
604 /*------------------------------------------------------------------------*/
605 
606 /*
607  * These functions manage primitive access to the RWConflict pool and lists.
608  */
609 static bool
611 {
612  dlist_iter iter;
613 
614  Assert(reader != writer);
615 
616  /* Check the ends of the purported conflict first. */
617  if (SxactIsDoomed(reader)
618  || SxactIsDoomed(writer)
619  || dlist_is_empty(&reader->outConflicts)
620  || dlist_is_empty(&writer->inConflicts))
621  return false;
622 
623  /*
624  * A conflict is possible; walk the list to find out.
625  *
626  * The unconstify is needed as we have no const version of
627  * dlist_foreach().
628  */
629  dlist_foreach(iter, &unconstify(SERIALIZABLEXACT *, reader)->outConflicts)
630  {
631  RWConflict conflict =
632  dlist_container(RWConflictData, outLink, iter.cur);
633 
634  if (conflict->sxactIn == writer)
635  return true;
636  }
637 
638  /* No conflict found. */
639  return false;
640 }
641 
642 static void
644 {
645  RWConflict conflict;
646 
647  Assert(reader != writer);
648  Assert(!RWConflictExists(reader, writer));
649 
651  ereport(ERROR,
652  (errcode(ERRCODE_OUT_OF_MEMORY),
653  errmsg("not enough elements in RWConflictPool to record a read/write conflict"),
654  errhint("You might need to run fewer transactions at a time or increase max_connections.")));
655 
657  dlist_delete(&conflict->outLink);
658 
659  conflict->sxactOut = reader;
660  conflict->sxactIn = writer;
661  dlist_push_tail(&reader->outConflicts, &conflict->outLink);
662  dlist_push_tail(&writer->inConflicts, &conflict->inLink);
663 }
664 
665 static void
667  SERIALIZABLEXACT *activeXact)
668 {
669  RWConflict conflict;
670 
671  Assert(roXact != activeXact);
672  Assert(SxactIsReadOnly(roXact));
673  Assert(!SxactIsReadOnly(activeXact));
674 
676  ereport(ERROR,
677  (errcode(ERRCODE_OUT_OF_MEMORY),
678  errmsg("not enough elements in RWConflictPool to record a potential read/write conflict"),
679  errhint("You might need to run fewer transactions at a time or increase max_connections.")));
680 
682  dlist_delete(&conflict->outLink);
683 
684  conflict->sxactOut = activeXact;
685  conflict->sxactIn = roXact;
686  dlist_push_tail(&activeXact->possibleUnsafeConflicts, &conflict->outLink);
687  dlist_push_tail(&roXact->possibleUnsafeConflicts, &conflict->inLink);
688 }
689 
690 static void
692 {
693  dlist_delete(&conflict->inLink);
694  dlist_delete(&conflict->outLink);
696 }
697 
698 static void
700 {
701  dlist_mutable_iter iter;
702 
703  Assert(SxactIsReadOnly(sxact));
704  Assert(!SxactIsROSafe(sxact));
705 
706  sxact->flags |= SXACT_FLAG_RO_UNSAFE;
707 
708  /*
709  * We know this isn't a safe snapshot, so we can stop looking for other
710  * potential conflicts.
711  */
713  {
714  RWConflict conflict =
715  dlist_container(RWConflictData, inLink, iter.cur);
716 
717  Assert(!SxactIsReadOnly(conflict->sxactOut));
718  Assert(sxact == conflict->sxactIn);
719 
720  ReleaseRWConflict(conflict);
721  }
722 }
723 
724 /*------------------------------------------------------------------------*/
725 
726 /*
727  * Decide whether a Serial page number is "older" for truncation purposes.
728  * Analogous to CLOGPagePrecedes().
729  */
730 static bool
731 SerialPagePrecedesLogically(int64 page1, int64 page2)
732 {
733  TransactionId xid1;
734  TransactionId xid2;
735 
736  xid1 = ((TransactionId) page1) * SERIAL_ENTRIESPERPAGE;
737  xid1 += FirstNormalTransactionId + 1;
738  xid2 = ((TransactionId) page2) * SERIAL_ENTRIESPERPAGE;
739  xid2 += FirstNormalTransactionId + 1;
740 
741  return (TransactionIdPrecedes(xid1, xid2) &&
742  TransactionIdPrecedes(xid1, xid2 + SERIAL_ENTRIESPERPAGE - 1));
743 }
744 
745 #ifdef USE_ASSERT_CHECKING
746 static void
747 SerialPagePrecedesLogicallyUnitTests(void)
748 {
749  int per_page = SERIAL_ENTRIESPERPAGE,
750  offset = per_page / 2;
751  int64 newestPage,
752  oldestPage,
753  headPage,
754  targetPage;
755  TransactionId newestXact,
756  oldestXact;
757 
758  /* GetNewTransactionId() has assigned the last XID it can safely use. */
759  newestPage = 2 * SLRU_PAGES_PER_SEGMENT - 1; /* nothing special */
760  newestXact = newestPage * per_page + offset;
761  Assert(newestXact / per_page == newestPage);
762  oldestXact = newestXact + 1;
763  oldestXact -= 1U << 31;
764  oldestPage = oldestXact / per_page;
765 
766  /*
767  * In this scenario, the SLRU headPage pertains to the last ~1000 XIDs
768  * assigned. oldestXact finishes, ~2B XIDs having elapsed since it
769  * started. Further transactions cause us to summarize oldestXact to
770  * tailPage. Function must return false so SerialAdd() doesn't zero
771  * tailPage (which may contain entries for other old, recently-finished
772  * XIDs) and half the SLRU. Reaching this requires burning ~2B XIDs in
773  * single-user mode, a negligible possibility.
774  */
775  headPage = newestPage;
776  targetPage = oldestPage;
778 
779  /*
780  * In this scenario, the SLRU headPage pertains to oldestXact. We're
781  * summarizing an XID near newestXact. (Assume few other XIDs used
782  * SERIALIZABLE, hence the minimal headPage advancement. Assume
783  * oldestXact was long-running and only recently reached the SLRU.)
784  * Function must return true to make SerialAdd() create targetPage.
785  *
786  * Today's implementation mishandles this case, but it doesn't matter
787  * enough to fix. Verify that the defect affects just one page by
788  * asserting correct treatment of its prior page. Reaching this case
789  * requires burning ~2B XIDs in single-user mode, a negligible
790  * possibility. Moreover, if it does happen, the consequence would be
791  * mild, namely a new transaction failing in SimpleLruReadPage().
792  */
793  headPage = oldestPage;
794  targetPage = newestPage;
795  Assert(SerialPagePrecedesLogically(headPage, targetPage - 1));
796 #if 0
798 #endif
799 }
800 #endif
801 
802 /*
803  * Initialize for the tracking of old serializable committed xids.
804  */
805 static void
807 {
808  bool found;
809 
810  /*
811  * Set up SLRU management of the pg_serial data.
812  */
814  SimpleLruInit(SerialSlruCtl, "serializable",
815  serializable_buffers, 0, "pg_serial",
817  SYNC_HANDLER_NONE, false);
818 #ifdef USE_ASSERT_CHECKING
819  SerialPagePrecedesLogicallyUnitTests();
820 #endif
822 
823  /*
824  * Create or attach to the SerialControl structure.
825  */
827  ShmemInitStruct("SerialControlData", sizeof(SerialControlData), &found);
828 
829  Assert(found == IsUnderPostmaster);
830  if (!found)
831  {
832  /*
833  * Set control information to reflect empty SLRU.
834  */
835  LWLockAcquire(SerialControlLock, LW_EXCLUSIVE);
836  serialControl->headPage = -1;
839  LWLockRelease(SerialControlLock);
840  }
841 }
842 
843 /*
844  * GUC check_hook for serializable_buffers
845  */
846 bool
848 {
849  return check_slru_buffers("serializable_buffers", newval);
850 }
851 
852 /*
853  * Record a committed read write serializable xid and the minimum
854  * commitSeqNo of any transactions to which this xid had a rw-conflict out.
855  * An invalid commitSeqNo means that there were no conflicts out from xid.
856  */
857 static void
858 SerialAdd(TransactionId xid, SerCommitSeqNo minConflictCommitSeqNo)
859 {
861  int64 targetPage;
862  int slotno;
863  int64 firstZeroPage;
864  bool isNewPage;
865  LWLock *lock;
866 
868 
869  targetPage = SerialPage(xid);
870  lock = SimpleLruGetBankLock(SerialSlruCtl, targetPage);
871 
872  /*
873  * In this routine, we must hold both SerialControlLock and the SLRU bank
874  * lock simultaneously while making the SLRU data catch up with the new
875  * state that we determine.
876  */
877  LWLockAcquire(SerialControlLock, LW_EXCLUSIVE);
878 
879  /*
880  * If no serializable transactions are active, there shouldn't be anything
881  * to push out to the SLRU. Hitting this assert would mean there's
882  * something wrong with the earlier cleanup logic.
883  */
886 
887  /*
888  * If the SLRU is currently unused, zero out the whole active region from
889  * tailXid to headXid before taking it into use. Otherwise zero out only
890  * any new pages that enter the tailXid-headXid range as we advance
891  * headXid.
892  */
893  if (serialControl->headPage < 0)
894  {
895  firstZeroPage = SerialPage(tailXid);
896  isNewPage = true;
897  }
898  else
899  {
900  firstZeroPage = SerialNextPage(serialControl->headPage);
902  targetPage);
903  }
904 
907  serialControl->headXid = xid;
908  if (isNewPage)
909  serialControl->headPage = targetPage;
910 
911  if (isNewPage)
912  {
913  /* Initialize intervening pages; might involve trading locks */
914  for (;;)
915  {
916  lock = SimpleLruGetBankLock(SerialSlruCtl, firstZeroPage);
918  slotno = SimpleLruZeroPage(SerialSlruCtl, firstZeroPage);
919  if (firstZeroPage == targetPage)
920  break;
921  firstZeroPage = SerialNextPage(firstZeroPage);
922  LWLockRelease(lock);
923  }
924  }
925  else
926  {
928  slotno = SimpleLruReadPage(SerialSlruCtl, targetPage, true, xid);
929  }
930 
931  SerialValue(slotno, xid) = minConflictCommitSeqNo;
932  SerialSlruCtl->shared->page_dirty[slotno] = true;
933 
934  LWLockRelease(lock);
935  LWLockRelease(SerialControlLock);
936 }
937 
938 /*
939  * Get the minimum commitSeqNo for any conflict out for the given xid. For
940  * a transaction which exists but has no conflict out, InvalidSerCommitSeqNo
941  * will be returned.
942  */
943 static SerCommitSeqNo
945 {
949  int slotno;
950 
952 
953  LWLockAcquire(SerialControlLock, LW_SHARED);
956  LWLockRelease(SerialControlLock);
957 
959  return 0;
960 
962 
964  || TransactionIdFollows(xid, headXid))
965  return 0;
966 
967  /*
968  * The following function must be called without holding SLRU bank lock,
969  * but will return with that lock held, which must then be released.
970  */
972  SerialPage(xid), xid);
973  val = SerialValue(slotno, xid);
975  return val;
976 }
977 
978 /*
979  * Call this whenever there is a new xmin for active serializable
980  * transactions. We don't need to keep information on transactions which
981  * precede that. InvalidTransactionId means none active, so everything in
982  * the SLRU can be discarded.
983  */
984 static void
986 {
987  LWLockAcquire(SerialControlLock, LW_EXCLUSIVE);
988 
989  /*
990  * When no sxacts are active, nothing overlaps, set the xid values to
991  * invalid to show that there are no valid entries. Don't clear headPage,
992  * though. A new xmin might still land on that page, and we don't want to
993  * repeatedly zero out the same page.
994  */
995  if (!TransactionIdIsValid(xid))
996  {
999  LWLockRelease(SerialControlLock);
1000  return;
1001  }
1002 
1003  /*
1004  * When we're recovering prepared transactions, the global xmin might move
1005  * backwards depending on the order they're recovered. Normally that's not
1006  * OK, but during recovery no serializable transactions will commit, so
1007  * the SLRU is empty and we can get away with it.
1008  */
1009  if (RecoveryInProgress())
1010  {
1014  {
1015  serialControl->tailXid = xid;
1016  }
1017  LWLockRelease(SerialControlLock);
1018  return;
1019  }
1020 
1023 
1024  serialControl->tailXid = xid;
1025 
1026  LWLockRelease(SerialControlLock);
1027 }
1028 
1029 /*
1030  * Perform a checkpoint --- either during shutdown, or on-the-fly
1031  *
1032  * We don't have any data that needs to survive a restart, but this is a
1033  * convenient place to truncate the SLRU.
1034  */
1035 void
1037 {
1038  int truncateCutoffPage;
1039 
1040  LWLockAcquire(SerialControlLock, LW_EXCLUSIVE);
1041 
1042  /* Exit quickly if the SLRU is currently not in use. */
1043  if (serialControl->headPage < 0)
1044  {
1045  LWLockRelease(SerialControlLock);
1046  return;
1047  }
1048 
1050  {
1051  int tailPage;
1052 
1053  tailPage = SerialPage(serialControl->tailXid);
1054 
1055  /*
1056  * It is possible for the tailXid to be ahead of the headXid. This
1057  * occurs if we checkpoint while there are in-progress serializable
1058  * transaction(s) advancing the tail but we are yet to summarize the
1059  * transactions. In this case, we cutoff up to the headPage and the
1060  * next summary will advance the headXid.
1061  */
1063  {
1064  /* We can truncate the SLRU up to the page containing tailXid */
1065  truncateCutoffPage = tailPage;
1066  }
1067  else
1068  truncateCutoffPage = serialControl->headPage;
1069  }
1070  else
1071  {
1072  /*----------
1073  * The SLRU is no longer needed. Truncate to head before we set head
1074  * invalid.
1075  *
1076  * XXX: It's possible that the SLRU is not needed again until XID
1077  * wrap-around has happened, so that the segment containing headPage
1078  * that we leave behind will appear to be new again. In that case it
1079  * won't be removed until XID horizon advances enough to make it
1080  * current again.
1081  *
1082  * XXX: This should happen in vac_truncate_clog(), not in checkpoints.
1083  * Consider this scenario, starting from a system with no in-progress
1084  * transactions and VACUUM FREEZE having maximized oldestXact:
1085  * - Start a SERIALIZABLE transaction.
1086  * - Start, finish, and summarize a SERIALIZABLE transaction, creating
1087  * one SLRU page.
1088  * - Consume XIDs to reach xidStopLimit.
1089  * - Finish all transactions. Due to the long-running SERIALIZABLE
1090  * transaction, earlier checkpoints did not touch headPage. The
1091  * next checkpoint will change it, but that checkpoint happens after
1092  * the end of the scenario.
1093  * - VACUUM to advance XID limits.
1094  * - Consume ~2M XIDs, crossing the former xidWrapLimit.
1095  * - Start, finish, and summarize a SERIALIZABLE transaction.
1096  * SerialAdd() declines to create the targetPage, because headPage
1097  * is not regarded as in the past relative to that targetPage. The
1098  * transaction instigating the summarize fails in
1099  * SimpleLruReadPage().
1100  */
1101  truncateCutoffPage = serialControl->headPage;
1102  serialControl->headPage = -1;
1103  }
1104 
1105  LWLockRelease(SerialControlLock);
1106 
1107  /*
1108  * Truncate away pages that are no longer required. Note that no
1109  * additional locking is required, because this is only called as part of
1110  * a checkpoint, and the validity limits have already been determined.
1111  */
1112  SimpleLruTruncate(SerialSlruCtl, truncateCutoffPage);
1113 
1114  /*
1115  * Write dirty SLRU pages to disk
1116  *
1117  * This is not actually necessary from a correctness point of view. We do
1118  * it merely as a debugging aid.
1119  *
1120  * We're doing this after the truncation to avoid writing pages right
1121  * before deleting the file in which they sit, which would be completely
1122  * pointless.
1123  */
1125 }
1126 
1127 /*------------------------------------------------------------------------*/
1128 
1129 /*
1130  * InitPredicateLocks -- Initialize the predicate locking data structures.
1131  *
1132  * This is called from CreateSharedMemoryAndSemaphores(), which see for
1133  * more comments. In the normal postmaster case, the shared hash tables
1134  * are created here. Backends inherit the pointers
1135  * to the shared tables via fork(). In the EXEC_BACKEND case, each
1136  * backend re-executes this code to obtain pointers to the already existing
1137  * shared hash tables.
1138  */
1139 void
1141 {
1142  HASHCTL info;
1143  long max_table_size;
1144  Size requestSize;
1145  bool found;
1146 
1147 #ifndef EXEC_BACKEND
1149 #endif
1150 
1151  /*
1152  * Compute size of predicate lock target hashtable. Note these
1153  * calculations must agree with PredicateLockShmemSize!
1154  */
1155  max_table_size = NPREDICATELOCKTARGETENTS();
1156 
1157  /*
1158  * Allocate hash table for PREDICATELOCKTARGET structs. This stores
1159  * per-predicate-lock-target information.
1160  */
1161  info.keysize = sizeof(PREDICATELOCKTARGETTAG);
1162  info.entrysize = sizeof(PREDICATELOCKTARGET);
1164 
1165  PredicateLockTargetHash = ShmemInitHash("PREDICATELOCKTARGET hash",
1166  max_table_size,
1167  max_table_size,
1168  &info,
1169  HASH_ELEM | HASH_BLOBS |
1171 
1172  /*
1173  * Reserve a dummy entry in the hash table; we use it to make sure there's
1174  * always one entry available when we need to split or combine a page,
1175  * because running out of space there could mean aborting a
1176  * non-serializable transaction.
1177  */
1178  if (!IsUnderPostmaster)
1179  {
1181  HASH_ENTER, &found);
1182  Assert(!found);
1183  }
1184 
1185  /* Pre-calculate the hash and partition lock of the scratch entry */
1188 
1189  /*
1190  * Allocate hash table for PREDICATELOCK structs. This stores per
1191  * xact-lock-of-a-target information.
1192  */
1193  info.keysize = sizeof(PREDICATELOCKTAG);
1194  info.entrysize = sizeof(PREDICATELOCK);
1195  info.hash = predicatelock_hash;
1197 
1198  /* Assume an average of 2 xacts per target */
1199  max_table_size *= 2;
1200 
1201  PredicateLockHash = ShmemInitHash("PREDICATELOCK hash",
1202  max_table_size,
1203  max_table_size,
1204  &info,
1207 
1208  /*
1209  * Compute size for serializable transaction hashtable. Note these
1210  * calculations must agree with PredicateLockShmemSize!
1211  */
1212  max_table_size = (MaxBackends + max_prepared_xacts);
1213 
1214  /*
1215  * Allocate a list to hold information on transactions participating in
1216  * predicate locking.
1217  *
1218  * Assume an average of 10 predicate locking transactions per backend.
1219  * This allows aggressive cleanup while detail is present before data must
1220  * be summarized for storage in SLRU and the "dummy" transaction.
1221  */
1222  max_table_size *= 10;
1223 
1224  PredXact = ShmemInitStruct("PredXactList",
1226  &found);
1227  Assert(found == IsUnderPostmaster);
1228  if (!found)
1229  {
1230  int i;
1231 
1240  requestSize = mul_size((Size) max_table_size,
1241  sizeof(SERIALIZABLEXACT));
1242  PredXact->element = ShmemAlloc(requestSize);
1243  /* Add all elements to available list, clean. */
1244  memset(PredXact->element, 0, requestSize);
1245  for (i = 0; i < max_table_size; i++)
1246  {
1250  }
1267  }
1268  /* This never changes, so let's keep a local copy. */
1270 
1271  /*
1272  * Allocate hash table for SERIALIZABLEXID structs. This stores per-xid
1273  * information for serializable transactions which have accessed data.
1274  */
1275  info.keysize = sizeof(SERIALIZABLEXIDTAG);
1276  info.entrysize = sizeof(SERIALIZABLEXID);
1277 
1278  SerializableXidHash = ShmemInitHash("SERIALIZABLEXID hash",
1279  max_table_size,
1280  max_table_size,
1281  &info,
1282  HASH_ELEM | HASH_BLOBS |
1283  HASH_FIXED_SIZE);
1284 
1285  /*
1286  * Allocate space for tracking rw-conflicts in lists attached to the
1287  * transactions.
1288  *
1289  * Assume an average of 5 conflicts per transaction. Calculations suggest
1290  * that this will prevent resource exhaustion in even the most pessimal
1291  * loads up to max_connections = 200 with all 200 connections pounding the
1292  * database with serializable transactions. Beyond that, there may be
1293  * occasional transactions canceled when trying to flag conflicts. That's
1294  * probably OK.
1295  */
1296  max_table_size *= 5;
1297 
1298  RWConflictPool = ShmemInitStruct("RWConflictPool",
1300  &found);
1301  Assert(found == IsUnderPostmaster);
1302  if (!found)
1303  {
1304  int i;
1305 
1307  requestSize = mul_size((Size) max_table_size,
1309  RWConflictPool->element = ShmemAlloc(requestSize);
1310  /* Add all elements to available list, clean. */
1311  memset(RWConflictPool->element, 0, requestSize);
1312  for (i = 0; i < max_table_size; i++)
1313  {
1316  }
1317  }
1318 
1319  /*
1320  * Create or attach to the header for the list of finished serializable
1321  * transactions.
1322  */
1324  ShmemInitStruct("FinishedSerializableTransactions",
1325  sizeof(dlist_head),
1326  &found);
1327  Assert(found == IsUnderPostmaster);
1328  if (!found)
1330 
1331  /*
1332  * Initialize the SLRU storage for old committed serializable
1333  * transactions.
1334  */
1335  SerialInit();
1336 }
1337 
1338 /*
1339  * Estimate shared-memory space used for predicate lock table
1340  */
1341 Size
1343 {
1344  Size size = 0;
1345  long max_table_size;
1346 
1347  /* predicate lock target hash table */
1348  max_table_size = NPREDICATELOCKTARGETENTS();
1349  size = add_size(size, hash_estimate_size(max_table_size,
1350  sizeof(PREDICATELOCKTARGET)));
1351 
1352  /* predicate lock hash table */
1353  max_table_size *= 2;
1354  size = add_size(size, hash_estimate_size(max_table_size,
1355  sizeof(PREDICATELOCK)));
1356 
1357  /*
1358  * Since NPREDICATELOCKTARGETENTS is only an estimate, add 10% safety
1359  * margin.
1360  */
1361  size = add_size(size, size / 10);
1362 
1363  /* transaction list */
1364  max_table_size = MaxBackends + max_prepared_xacts;
1365  max_table_size *= 10;
1367  size = add_size(size, mul_size((Size) max_table_size,
1368  sizeof(SERIALIZABLEXACT)));
1369 
1370  /* transaction xid table */
1371  size = add_size(size, hash_estimate_size(max_table_size,
1372  sizeof(SERIALIZABLEXID)));
1373 
1374  /* rw-conflict pool */
1375  max_table_size *= 5;
1377  size = add_size(size, mul_size((Size) max_table_size,
1379 
1380  /* Head for list of finished serializable transactions. */
1381  size = add_size(size, sizeof(dlist_head));
1382 
1383  /* Shared memory structures for SLRU tracking of old committed xids. */
1384  size = add_size(size, sizeof(SerialControlData));
1386 
1387  return size;
1388 }
1389 
1390 
1391 /*
1392  * Compute the hash code associated with a PREDICATELOCKTAG.
1393  *
1394  * Because we want to use just one set of partition locks for both the
1395  * PREDICATELOCKTARGET and PREDICATELOCK hash tables, we have to make sure
1396  * that PREDICATELOCKs fall into the same partition number as their
1397  * associated PREDICATELOCKTARGETs. dynahash.c expects the partition number
1398  * to be the low-order bits of the hash code, and therefore a
1399  * PREDICATELOCKTAG's hash code must have the same low-order bits as the
1400  * associated PREDICATELOCKTARGETTAG's hash code. We achieve this with this
1401  * specialized hash function.
1402  */
1403 static uint32
1404 predicatelock_hash(const void *key, Size keysize)
1405 {
1406  const PREDICATELOCKTAG *predicatelocktag = (const PREDICATELOCKTAG *) key;
1407  uint32 targethash;
1408 
1409  Assert(keysize == sizeof(PREDICATELOCKTAG));
1410 
1411  /* Look into the associated target object, and compute its hash code */
1412  targethash = PredicateLockTargetTagHashCode(&predicatelocktag->myTarget->tag);
1413 
1414  return PredicateLockHashCodeFromTargetHashCode(predicatelocktag, targethash);
1415 }
1416 
1417 
1418 /*
1419  * GetPredicateLockStatusData
1420  * Return a table containing the internal state of the predicate
1421  * lock manager for use in pg_lock_status.
1422  *
1423  * Like GetLockStatusData, this function tries to hold the partition LWLocks
1424  * for as short a time as possible by returning two arrays that simply
1425  * contain the PREDICATELOCKTARGETTAG and SERIALIZABLEXACT for each lock
1426  * table entry. Multiple copies of the same PREDICATELOCKTARGETTAG and
1427  * SERIALIZABLEXACT will likely appear.
1428  */
1431 {
1433  int i;
1434  int els,
1435  el;
1436  HASH_SEQ_STATUS seqstat;
1437  PREDICATELOCK *predlock;
1438 
1440 
1441  /*
1442  * To ensure consistency, take simultaneous locks on all partition locks
1443  * in ascending order, then SerializableXactHashLock.
1444  */
1445  for (i = 0; i < NUM_PREDICATELOCK_PARTITIONS; i++)
1447  LWLockAcquire(SerializableXactHashLock, LW_SHARED);
1448 
1449  /* Get number of locks and allocate appropriately-sized arrays. */
1451  data->nelements = els;
1452  data->locktags = (PREDICATELOCKTARGETTAG *)
1453  palloc(sizeof(PREDICATELOCKTARGETTAG) * els);
1454  data->xacts = (SERIALIZABLEXACT *)
1455  palloc(sizeof(SERIALIZABLEXACT) * els);
1456 
1457 
1458  /* Scan through PredicateLockHash and copy contents */
1459  hash_seq_init(&seqstat, PredicateLockHash);
1460 
1461  el = 0;
1462 
1463  while ((predlock = (PREDICATELOCK *) hash_seq_search(&seqstat)))
1464  {
1465  data->locktags[el] = predlock->tag.myTarget->tag;
1466  data->xacts[el] = *predlock->tag.myXact;
1467  el++;
1468  }
1469 
1470  Assert(el == els);
1471 
1472  /* Release locks in reverse order */
1473  LWLockRelease(SerializableXactHashLock);
1474  for (i = NUM_PREDICATELOCK_PARTITIONS - 1; i >= 0; i--)
1476 
1477  return data;
1478 }
1479 
1480 /*
1481  * Free up shared memory structures by pushing the oldest sxact (the one at
1482  * the front of the SummarizeOldestCommittedSxact queue) into summary form.
1483  * Each call will free exactly one SERIALIZABLEXACT structure and may also
1484  * free one or more of these structures: SERIALIZABLEXID, PREDICATELOCK,
1485  * PREDICATELOCKTARGET, RWConflictData.
1486  */
1487 static void
1489 {
1490  SERIALIZABLEXACT *sxact;
1491 
1492  LWLockAcquire(SerializableFinishedListLock, LW_EXCLUSIVE);
1493 
1494  /*
1495  * This function is only called if there are no sxact slots available.
1496  * Some of them must belong to old, already-finished transactions, so
1497  * there should be something in FinishedSerializableTransactions list that
1498  * we can summarize. However, there's a race condition: while we were not
1499  * holding any locks, a transaction might have ended and cleaned up all
1500  * the finished sxact entries already, freeing up their sxact slots. In
1501  * that case, we have nothing to do here. The caller will find one of the
1502  * slots released by the other backend when it retries.
1503  */
1505  {
1506  LWLockRelease(SerializableFinishedListLock);
1507  return;
1508  }
1509 
1510  /*
1511  * Grab the first sxact off the finished list -- this will be the earliest
1512  * commit. Remove it from the list.
1513  */
1514  sxact = dlist_head_element(SERIALIZABLEXACT, finishedLink,
1517 
1518  /* Add to SLRU summary information. */
1519  if (TransactionIdIsValid(sxact->topXid) && !SxactIsReadOnly(sxact))
1520  SerialAdd(sxact->topXid, SxactHasConflictOut(sxact)
1522 
1523  /* Summarize and release the detail. */
1524  ReleaseOneSerializableXact(sxact, false, true);
1525 
1526  LWLockRelease(SerializableFinishedListLock);
1527 }
1528 
1529 /*
1530  * GetSafeSnapshot
1531  * Obtain and register a snapshot for a READ ONLY DEFERRABLE
1532  * transaction. Ensures that the snapshot is "safe", i.e. a
1533  * read-only transaction running on it can execute serializably
1534  * without further checks. This requires waiting for concurrent
1535  * transactions to complete, and retrying with a new snapshot if
1536  * one of them could possibly create a conflict.
1537  *
1538  * As with GetSerializableTransactionSnapshot (which this is a subroutine
1539  * for), the passed-in Snapshot pointer should reference a static data
1540  * area that can safely be passed to GetSnapshotData.
1541  */
1542 static Snapshot
1544 {
1545  Snapshot snapshot;
1546 
1548 
1549  while (true)
1550  {
1551  /*
1552  * GetSerializableTransactionSnapshotInt is going to call
1553  * GetSnapshotData, so we need to provide it the static snapshot area
1554  * our caller passed to us. The pointer returned is actually the same
1555  * one passed to it, but we avoid assuming that here.
1556  */
1557  snapshot = GetSerializableTransactionSnapshotInt(origSnapshot,
1558  NULL, InvalidPid);
1559 
1561  return snapshot; /* no concurrent r/w xacts; it's safe */
1562 
1563  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
1564 
1565  /*
1566  * Wait for concurrent transactions to finish. Stop early if one of
1567  * them marked us as conflicted.
1568  */
1572  {
1573  LWLockRelease(SerializableXactHashLock);
1574  ProcWaitForSignal(WAIT_EVENT_SAFE_SNAPSHOT);
1575  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
1576  }
1578 
1580  {
1581  LWLockRelease(SerializableXactHashLock);
1582  break; /* success */
1583  }
1584 
1585  LWLockRelease(SerializableXactHashLock);
1586 
1587  /* else, need to retry... */
1588  ereport(DEBUG2,
1590  errmsg_internal("deferrable snapshot was unsafe; trying a new one")));
1591  ReleasePredicateLocks(false, false);
1592  }
1593 
1594  /*
1595  * Now we have a safe snapshot, so we don't need to do any further checks.
1596  */
1598  ReleasePredicateLocks(false, true);
1599 
1600  return snapshot;
1601 }
1602 
1603 /*
1604  * GetSafeSnapshotBlockingPids
1605  * If the specified process is currently blocked in GetSafeSnapshot,
1606  * write the process IDs of all processes that it is blocked by
1607  * into the caller-supplied buffer output[]. The list is truncated at
1608  * output_size, and the number of PIDs written into the buffer is
1609  * returned. Returns zero if the given PID is not currently blocked
1610  * in GetSafeSnapshot.
1611  */
1612 int
1613 GetSafeSnapshotBlockingPids(int blocked_pid, int *output, int output_size)
1614 {
1615  int num_written = 0;
1616  dlist_iter iter;
1617  SERIALIZABLEXACT *blocking_sxact = NULL;
1618 
1619  LWLockAcquire(SerializableXactHashLock, LW_SHARED);
1620 
1621  /* Find blocked_pid's SERIALIZABLEXACT by linear search. */
1623  {
1624  SERIALIZABLEXACT *sxact =
1625  dlist_container(SERIALIZABLEXACT, xactLink, iter.cur);
1626 
1627  if (sxact->pid == blocked_pid)
1628  {
1629  blocking_sxact = sxact;
1630  break;
1631  }
1632  }
1633 
1634  /* Did we find it, and is it currently waiting in GetSafeSnapshot? */
1635  if (blocking_sxact != NULL && SxactIsDeferrableWaiting(blocking_sxact))
1636  {
1637  /* Traverse the list of possible unsafe conflicts collecting PIDs. */
1638  dlist_foreach(iter, &blocking_sxact->possibleUnsafeConflicts)
1639  {
1640  RWConflict possibleUnsafeConflict =
1641  dlist_container(RWConflictData, inLink, iter.cur);
1642 
1643  output[num_written++] = possibleUnsafeConflict->sxactOut->pid;
1644 
1645  if (num_written >= output_size)
1646  break;
1647  }
1648  }
1649 
1650  LWLockRelease(SerializableXactHashLock);
1651 
1652  return num_written;
1653 }
1654 
1655 /*
1656  * Acquire a snapshot that can be used for the current transaction.
1657  *
1658  * Make sure we have a SERIALIZABLEXACT reference in MySerializableXact.
1659  * It should be current for this process and be contained in PredXact.
1660  *
1661  * The passed-in Snapshot pointer should reference a static data area that
1662  * can safely be passed to GetSnapshotData. The return value is actually
1663  * always this same pointer; no new snapshot data structure is allocated
1664  * within this function.
1665  */
1666 Snapshot
1668 {
1670 
1671  /*
1672  * Can't use serializable mode while recovery is still active, as it is,
1673  * for example, on a hot standby. We could get here despite the check in
1674  * check_transaction_isolation() if default_transaction_isolation is set
1675  * to serializable, so phrase the hint accordingly.
1676  */
1677  if (RecoveryInProgress())
1678  ereport(ERROR,
1679  (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
1680  errmsg("cannot use serializable mode in a hot standby"),
1681  errdetail("default_transaction_isolation is set to \"serializable\"."),
1682  errhint("You can use \"SET default_transaction_isolation = 'repeatable read'\" to change the default.")));
1683 
1684  /*
1685  * A special optimization is available for SERIALIZABLE READ ONLY
1686  * DEFERRABLE transactions -- we can wait for a suitable snapshot and
1687  * thereby avoid all SSI overhead once it's running.
1688  */
1690  return GetSafeSnapshot(snapshot);
1691 
1692  return GetSerializableTransactionSnapshotInt(snapshot,
1693  NULL, InvalidPid);
1694 }
1695 
1696 /*
1697  * Import a snapshot to be used for the current transaction.
1698  *
1699  * This is nearly the same as GetSerializableTransactionSnapshot, except that
1700  * we don't take a new snapshot, but rather use the data we're handed.
1701  *
1702  * The caller must have verified that the snapshot came from a serializable
1703  * transaction; and if we're read-write, the source transaction must not be
1704  * read-only.
1705  */
1706 void
1708  VirtualTransactionId *sourcevxid,
1709  int sourcepid)
1710 {
1712 
1713  /*
1714  * If this is called by parallel.c in a parallel worker, we don't want to
1715  * create a SERIALIZABLEXACT just yet because the leader's
1716  * SERIALIZABLEXACT will be installed with AttachSerializableXact(). We
1717  * also don't want to reject SERIALIZABLE READ ONLY DEFERRABLE in this
1718  * case, because the leader has already determined that the snapshot it
1719  * has passed us is safe. So there is nothing for us to do.
1720  */
1721  if (IsParallelWorker())
1722  return;
1723 
1724  /*
1725  * We do not allow SERIALIZABLE READ ONLY DEFERRABLE transactions to
1726  * import snapshots, since there's no way to wait for a safe snapshot when
1727  * we're using the snap we're told to. (XXX instead of throwing an error,
1728  * we could just ignore the XactDeferrable flag?)
1729  */
1731  ereport(ERROR,
1732  (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
1733  errmsg("a snapshot-importing transaction must not be READ ONLY DEFERRABLE")));
1734 
1735  (void) GetSerializableTransactionSnapshotInt(snapshot, sourcevxid,
1736  sourcepid);
1737 }
1738 
1739 /*
1740  * Guts of GetSerializableTransactionSnapshot
1741  *
1742  * If sourcevxid is valid, this is actually an import operation and we should
1743  * skip calling GetSnapshotData, because the snapshot contents are already
1744  * loaded up. HOWEVER: to avoid race conditions, we must check that the
1745  * source xact is still running after we acquire SerializableXactHashLock.
1746  * We do that by calling ProcArrayInstallImportedXmin.
1747  */
1748 static Snapshot
1750  VirtualTransactionId *sourcevxid,
1751  int sourcepid)
1752 {
1753  PGPROC *proc;
1754  VirtualTransactionId vxid;
1755  SERIALIZABLEXACT *sxact,
1756  *othersxact;
1757 
1758  /* We only do this for serializable transactions. Once. */
1760 
1762 
1763  /*
1764  * Since all parts of a serializable transaction must use the same
1765  * snapshot, it is too late to establish one after a parallel operation
1766  * has begun.
1767  */
1768  if (IsInParallelMode())
1769  elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
1770 
1771  proc = MyProc;
1772  Assert(proc != NULL);
1773  GET_VXID_FROM_PGPROC(vxid, *proc);
1774 
1775  /*
1776  * First we get the sxact structure, which may involve looping and access
1777  * to the "finished" list to free a structure for use.
1778  *
1779  * We must hold SerializableXactHashLock when taking/checking the snapshot
1780  * to avoid race conditions, for much the same reasons that
1781  * GetSnapshotData takes the ProcArrayLock. Since we might have to
1782  * release SerializableXactHashLock to call SummarizeOldestCommittedSxact,
1783  * this means we have to create the sxact first, which is a bit annoying
1784  * (in particular, an elog(ERROR) in procarray.c would cause us to leak
1785  * the sxact). Consider refactoring to avoid this.
1786  */
1787 #ifdef TEST_SUMMARIZE_SERIAL
1789 #endif
1790  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
1791  do
1792  {
1793  sxact = CreatePredXact();
1794  /* If null, push out committed sxact to SLRU summary & retry. */
1795  if (!sxact)
1796  {
1797  LWLockRelease(SerializableXactHashLock);
1799  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
1800  }
1801  } while (!sxact);
1802 
1803  /* Get the snapshot, or check that it's safe to use */
1804  if (!sourcevxid)
1805  snapshot = GetSnapshotData(snapshot);
1806  else if (!ProcArrayInstallImportedXmin(snapshot->xmin, sourcevxid))
1807  {
1808  ReleasePredXact(sxact);
1809  LWLockRelease(SerializableXactHashLock);
1810  ereport(ERROR,
1811  (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
1812  errmsg("could not import the requested snapshot"),
1813  errdetail("The source process with PID %d is not running anymore.",
1814  sourcepid)));
1815  }
1816 
1817  /*
1818  * If there are no serializable transactions which are not read-only, we
1819  * can "opt out" of predicate locking and conflict checking for a
1820  * read-only transaction.
1821  *
1822  * The reason this is safe is that a read-only transaction can only become
1823  * part of a dangerous structure if it overlaps a writable transaction
1824  * which in turn overlaps a writable transaction which committed before
1825  * the read-only transaction started. A new writable transaction can
1826  * overlap this one, but it can't meet the other condition of overlapping
1827  * a transaction which committed before this one started.
1828  */
1830  {
1831  ReleasePredXact(sxact);
1832  LWLockRelease(SerializableXactHashLock);
1833  return snapshot;
1834  }
1835 
1836  /* Initialize the structure. */
1837  sxact->vxid = vxid;
1841  dlist_init(&(sxact->outConflicts));
1842  dlist_init(&(sxact->inConflicts));
1844  sxact->topXid = GetTopTransactionIdIfAny();
1846  sxact->xmin = snapshot->xmin;
1847  sxact->pid = MyProcPid;
1848  sxact->pgprocno = MyProcNumber;
1849  dlist_init(&sxact->predicateLocks);
1850  dlist_node_init(&sxact->finishedLink);
1851  sxact->flags = 0;
1852  if (XactReadOnly)
1853  {
1854  dlist_iter iter;
1855 
1856  sxact->flags |= SXACT_FLAG_READ_ONLY;
1857 
1858  /*
1859  * Register all concurrent r/w transactions as possible conflicts; if
1860  * all of them commit without any outgoing conflicts to earlier
1861  * transactions then this snapshot can be deemed safe (and we can run
1862  * without tracking predicate locks).
1863  */
1865  {
1866  othersxact = dlist_container(SERIALIZABLEXACT, xactLink, iter.cur);
1867 
1868  if (!SxactIsCommitted(othersxact)
1869  && !SxactIsDoomed(othersxact)
1870  && !SxactIsReadOnly(othersxact))
1871  {
1872  SetPossibleUnsafeConflict(sxact, othersxact);
1873  }
1874  }
1875 
1876  /*
1877  * If we didn't find any possibly unsafe conflicts because every
1878  * uncommitted writable transaction turned out to be doomed, then we
1879  * can "opt out" immediately. See comments above the earlier check
1880  * for PredXact->WritableSxactCount == 0.
1881  */
1883  {
1884  ReleasePredXact(sxact);
1885  LWLockRelease(SerializableXactHashLock);
1886  return snapshot;
1887  }
1888  }
1889  else
1890  {
1894  }
1895 
1896  /* Maintain serializable global xmin info. */
1898  {
1900  PredXact->SxactGlobalXmin = snapshot->xmin;
1902  SerialSetActiveSerXmin(snapshot->xmin);
1903  }
1904  else if (TransactionIdEquals(snapshot->xmin, PredXact->SxactGlobalXmin))
1905  {
1908  }
1909  else
1910  {
1912  }
1913 
1914  MySerializableXact = sxact;
1915  MyXactDidWrite = false; /* haven't written anything yet */
1916 
1917  LWLockRelease(SerializableXactHashLock);
1918 
1920 
1921  return snapshot;
1922 }
1923 
1924 static void
1926 {
1927  HASHCTL hash_ctl;
1928 
1929  /* Initialize the backend-local hash table of parent locks */
1930  Assert(LocalPredicateLockHash == NULL);
1931  hash_ctl.keysize = sizeof(PREDICATELOCKTARGETTAG);
1932  hash_ctl.entrysize = sizeof(LOCALPREDICATELOCK);
1933  LocalPredicateLockHash = hash_create("Local predicate lock",
1935  &hash_ctl,
1936  HASH_ELEM | HASH_BLOBS);
1937 }
1938 
1939 /*
1940  * Register the top level XID in SerializableXidHash.
1941  * Also store it for easy reference in MySerializableXact.
1942  */
1943 void
1945 {
1946  SERIALIZABLEXIDTAG sxidtag;
1947  SERIALIZABLEXID *sxid;
1948  bool found;
1949 
1950  /*
1951  * If we're not tracking predicate lock data for this transaction, we
1952  * should ignore the request and return quickly.
1953  */
1955  return;
1956 
1957  /* We should have a valid XID and be at the top level. */
1959 
1960  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
1961 
1962  /* This should only be done once per transaction. */
1964 
1965  MySerializableXact->topXid = xid;
1966 
1967  sxidtag.xid = xid;
1969  &sxidtag,
1970  HASH_ENTER, &found);
1971  Assert(!found);
1972 
1973  /* Initialize the structure. */
1974  sxid->myXact = MySerializableXact;
1975  LWLockRelease(SerializableXactHashLock);
1976 }
1977 
1978 
1979 /*
1980  * Check whether there are any predicate locks held by any transaction
1981  * for the page at the given block number.
1982  *
1983  * Note that the transaction may be completed but not yet subject to
1984  * cleanup due to overlapping serializable transactions. This must
1985  * return valid information regardless of transaction isolation level.
1986  *
1987  * Also note that this doesn't check for a conflicting relation lock,
1988  * just a lock specifically on the given page.
1989  *
1990  * One use is to support proper behavior during GiST index vacuum.
1991  */
1992 bool
1994 {
1995  PREDICATELOCKTARGETTAG targettag;
1996  uint32 targettaghash;
1997  LWLock *partitionLock;
1998  PREDICATELOCKTARGET *target;
1999 
2001  relation->rd_locator.dbOid,
2002  relation->rd_id,
2003  blkno);
2004 
2005  targettaghash = PredicateLockTargetTagHashCode(&targettag);
2006  partitionLock = PredicateLockHashPartitionLock(targettaghash);
2007  LWLockAcquire(partitionLock, LW_SHARED);
2008  target = (PREDICATELOCKTARGET *)
2010  &targettag, targettaghash,
2011  HASH_FIND, NULL);
2012  LWLockRelease(partitionLock);
2013 
2014  return (target != NULL);
2015 }
2016 
2017 
2018 /*
2019  * Check whether a particular lock is held by this transaction.
2020  *
2021  * Important note: this function may return false even if the lock is
2022  * being held, because it uses the local lock table which is not
2023  * updated if another transaction modifies our lock list (e.g. to
2024  * split an index page). It can also return true when a coarser
2025  * granularity lock that covers this target is being held. Be careful
2026  * to only use this function in circumstances where such errors are
2027  * acceptable!
2028  */
2029 static bool
2031 {
2032  LOCALPREDICATELOCK *lock;
2033 
2034  /* check local hash table */
2036  targettag,
2037  HASH_FIND, NULL);
2038 
2039  if (!lock)
2040  return false;
2041 
2042  /*
2043  * Found entry in the table, but still need to check whether it's actually
2044  * held -- it could just be a parent of some held lock.
2045  */
2046  return lock->held;
2047 }
2048 
2049 /*
2050  * Return the parent lock tag in the lock hierarchy: the next coarser
2051  * lock that covers the provided tag.
2052  *
2053  * Returns true and sets *parent to the parent tag if one exists,
2054  * returns false if none exists.
2055  */
2056 static bool
2058  PREDICATELOCKTARGETTAG *parent)
2059 {
2060  switch (GET_PREDICATELOCKTARGETTAG_TYPE(*tag))
2061  {
2062  case PREDLOCKTAG_RELATION:
2063  /* relation locks have no parent lock */
2064  return false;
2065 
2066  case PREDLOCKTAG_PAGE:
2067  /* parent lock is relation lock */
2071 
2072  return true;
2073 
2074  case PREDLOCKTAG_TUPLE:
2075  /* parent lock is page lock */
2080  return true;
2081  }
2082 
2083  /* not reachable */
2084  Assert(false);
2085  return false;
2086 }
2087 
2088 /*
2089  * Check whether the lock we are considering is already covered by a
2090  * coarser lock for our transaction.
2091  *
2092  * Like PredicateLockExists, this function might return a false
2093  * negative, but it will never return a false positive.
2094  */
2095 static bool
2097 {
2098  PREDICATELOCKTARGETTAG targettag,
2099  parenttag;
2100 
2101  targettag = *newtargettag;
2102 
2103  /* check parents iteratively until no more */
2104  while (GetParentPredicateLockTag(&targettag, &parenttag))
2105  {
2106  targettag = parenttag;
2107  if (PredicateLockExists(&targettag))
2108  return true;
2109  }
2110 
2111  /* no more parents to check; lock is not covered */
2112  return false;
2113 }
2114 
2115 /*
2116  * Remove the dummy entry from the predicate lock target hash, to free up some
2117  * scratch space. The caller must be holding SerializablePredicateListLock,
2118  * and must restore the entry with RestoreScratchTarget() before releasing the
2119  * lock.
2120  *
2121  * If lockheld is true, the caller is already holding the partition lock
2122  * of the partition containing the scratch entry.
2123  */
2124 static void
2125 RemoveScratchTarget(bool lockheld)
2126 {
2127  bool found;
2128 
2129  Assert(LWLockHeldByMe(SerializablePredicateListLock));
2130 
2131  if (!lockheld)
2136  HASH_REMOVE, &found);
2137  Assert(found);
2138  if (!lockheld)
2140 }
2141 
2142 /*
2143  * Re-insert the dummy entry in predicate lock target hash.
2144  */
2145 static void
2146 RestoreScratchTarget(bool lockheld)
2147 {
2148  bool found;
2149 
2150  Assert(LWLockHeldByMe(SerializablePredicateListLock));
2151 
2152  if (!lockheld)
2157  HASH_ENTER, &found);
2158  Assert(!found);
2159  if (!lockheld)
2161 }
2162 
2163 /*
2164  * Check whether the list of related predicate locks is empty for a
2165  * predicate lock target, and remove the target if it is.
2166  */
2167 static void
2169 {
2171 
2172  Assert(LWLockHeldByMe(SerializablePredicateListLock));
2173 
2174  /* Can't remove it until no locks at this target. */
2175  if (!dlist_is_empty(&target->predicateLocks))
2176  return;
2177 
2178  /* Actually remove the target. */
2180  &target->tag,
2181  targettaghash,
2182  HASH_REMOVE, NULL);
2183  Assert(rmtarget == target);
2184 }
2185 
2186 /*
2187  * Delete child target locks owned by this process.
2188  * This implementation is assuming that the usage of each target tag field
2189  * is uniform. No need to make this hard if we don't have to.
2190  *
2191  * We acquire an LWLock in the case of parallel mode, because worker
2192  * backends have access to the leader's SERIALIZABLEXACT. Otherwise,
2193  * we aren't acquiring LWLocks for the predicate lock or lock
2194  * target structures associated with this transaction unless we're going
2195  * to modify them, because no other process is permitted to modify our
2196  * locks.
2197  */
2198 static void
2200 {
2201  SERIALIZABLEXACT *sxact;
2202  PREDICATELOCK *predlock;
2203  dlist_mutable_iter iter;
2204 
2205  LWLockAcquire(SerializablePredicateListLock, LW_SHARED);
2206  sxact = MySerializableXact;
2207  if (IsInParallelMode())
2209 
2210  dlist_foreach_modify(iter, &sxact->predicateLocks)
2211  {
2212  PREDICATELOCKTAG oldlocktag;
2213  PREDICATELOCKTARGET *oldtarget;
2214  PREDICATELOCKTARGETTAG oldtargettag;
2215 
2216  predlock = dlist_container(PREDICATELOCK, xactLink, iter.cur);
2217 
2218  oldlocktag = predlock->tag;
2219  Assert(oldlocktag.myXact == sxact);
2220  oldtarget = oldlocktag.myTarget;
2221  oldtargettag = oldtarget->tag;
2222 
2223  if (TargetTagIsCoveredBy(oldtargettag, *newtargettag))
2224  {
2225  uint32 oldtargettaghash;
2226  LWLock *partitionLock;
2228 
2229  oldtargettaghash = PredicateLockTargetTagHashCode(&oldtargettag);
2230  partitionLock = PredicateLockHashPartitionLock(oldtargettaghash);
2231 
2232  LWLockAcquire(partitionLock, LW_EXCLUSIVE);
2233 
2234  dlist_delete(&predlock->xactLink);
2235  dlist_delete(&predlock->targetLink);
2236  rmpredlock = hash_search_with_hash_value
2238  &oldlocktag,
2240  oldtargettaghash),
2241  HASH_REMOVE, NULL);
2242  Assert(rmpredlock == predlock);
2243 
2244  RemoveTargetIfNoLongerUsed(oldtarget, oldtargettaghash);
2245 
2246  LWLockRelease(partitionLock);
2247 
2248  DecrementParentLocks(&oldtargettag);
2249  }
2250  }
2251  if (IsInParallelMode())
2253  LWLockRelease(SerializablePredicateListLock);
2254 }
2255 
2256 /*
2257  * Returns the promotion limit for a given predicate lock target. This is the
2258  * max number of descendant locks allowed before promoting to the specified
2259  * tag. Note that the limit includes non-direct descendants (e.g., both tuples
2260  * and pages for a relation lock).
2261  *
2262  * Currently the default limit is 2 for a page lock, and half of the value of
2263  * max_pred_locks_per_transaction - 1 for a relation lock, to match behavior
2264  * of earlier releases when upgrading.
2265  *
2266  * TODO SSI: We should probably add additional GUCs to allow a maximum ratio
2267  * of page and tuple locks based on the pages in a relation, and the maximum
2268  * ratio of tuple locks to tuples in a page. This would provide more
2269  * generally "balanced" allocation of locks to where they are most useful,
2270  * while still allowing the absolute numbers to prevent one relation from
2271  * tying up all predicate lock resources.
2272  */
2273 static int
2275 {
2276  switch (GET_PREDICATELOCKTARGETTAG_TYPE(*tag))
2277  {
2278  case PREDLOCKTAG_RELATION:
2283 
2284  case PREDLOCKTAG_PAGE:
2286 
2287  case PREDLOCKTAG_TUPLE:
2288 
2289  /*
2290  * not reachable: nothing is finer-granularity than a tuple, so we
2291  * should never try to promote to it.
2292  */
2293  Assert(false);
2294  return 0;
2295  }
2296 
2297  /* not reachable */
2298  Assert(false);
2299  return 0;
2300 }
2301 
2302 /*
2303  * For all ancestors of a newly-acquired predicate lock, increment
2304  * their child count in the parent hash table. If any of them have
2305  * more descendants than their promotion threshold, acquire the
2306  * coarsest such lock.
2307  *
2308  * Returns true if a parent lock was acquired and false otherwise.
2309  */
2310 static bool
2312 {
2313  PREDICATELOCKTARGETTAG targettag,
2314  nexttag,
2315  promotiontag;
2316  LOCALPREDICATELOCK *parentlock;
2317  bool found,
2318  promote;
2319 
2320  promote = false;
2321 
2322  targettag = *reqtag;
2323 
2324  /* check parents iteratively */
2325  while (GetParentPredicateLockTag(&targettag, &nexttag))
2326  {
2327  targettag = nexttag;
2329  &targettag,
2330  HASH_ENTER,
2331  &found);
2332  if (!found)
2333  {
2334  parentlock->held = false;
2335  parentlock->childLocks = 1;
2336  }
2337  else
2338  parentlock->childLocks++;
2339 
2340  if (parentlock->childLocks >
2341  MaxPredicateChildLocks(&targettag))
2342  {
2343  /*
2344  * We should promote to this parent lock. Continue to check its
2345  * ancestors, however, both to get their child counts right and to
2346  * check whether we should just go ahead and promote to one of
2347  * them.
2348  */
2349  promotiontag = targettag;
2350  promote = true;
2351  }
2352  }
2353 
2354  if (promote)
2355  {
2356  /* acquire coarsest ancestor eligible for promotion */
2357  PredicateLockAcquire(&promotiontag);
2358  return true;
2359  }
2360  else
2361  return false;
2362 }
2363 
2364 /*
2365  * When releasing a lock, decrement the child count on all ancestor
2366  * locks.
2367  *
2368  * This is called only when releasing a lock via
2369  * DeleteChildTargetLocks (i.e. when a lock becomes redundant because
2370  * we've acquired its parent, possibly due to promotion) or when a new
2371  * MVCC write lock makes the predicate lock unnecessary. There's no
2372  * point in calling it when locks are released at transaction end, as
2373  * this information is no longer needed.
2374  */
2375 static void
2377 {
2378  PREDICATELOCKTARGETTAG parenttag,
2379  nexttag;
2380 
2381  parenttag = *targettag;
2382 
2383  while (GetParentPredicateLockTag(&parenttag, &nexttag))
2384  {
2385  uint32 targettaghash;
2386  LOCALPREDICATELOCK *parentlock,
2387  *rmlock PG_USED_FOR_ASSERTS_ONLY;
2388 
2389  parenttag = nexttag;
2390  targettaghash = PredicateLockTargetTagHashCode(&parenttag);
2391  parentlock = (LOCALPREDICATELOCK *)
2393  &parenttag, targettaghash,
2394  HASH_FIND, NULL);
2395 
2396  /*
2397  * There's a small chance the parent lock doesn't exist in the lock
2398  * table. This can happen if we prematurely removed it because an
2399  * index split caused the child refcount to be off.
2400  */
2401  if (parentlock == NULL)
2402  continue;
2403 
2404  parentlock->childLocks--;
2405 
2406  /*
2407  * Under similar circumstances the parent lock's refcount might be
2408  * zero. This only happens if we're holding that lock (otherwise we
2409  * would have removed the entry).
2410  */
2411  if (parentlock->childLocks < 0)
2412  {
2413  Assert(parentlock->held);
2414  parentlock->childLocks = 0;
2415  }
2416 
2417  if ((parentlock->childLocks == 0) && (!parentlock->held))
2418  {
2419  rmlock = (LOCALPREDICATELOCK *)
2421  &parenttag, targettaghash,
2422  HASH_REMOVE, NULL);
2423  Assert(rmlock == parentlock);
2424  }
2425  }
2426 }
2427 
2428 /*
2429  * Indicate that a predicate lock on the given target is held by the
2430  * specified transaction. Has no effect if the lock is already held.
2431  *
2432  * This updates the lock table and the sxact's lock list, and creates
2433  * the lock target if necessary, but does *not* do anything related to
2434  * granularity promotion or the local lock table. See
2435  * PredicateLockAcquire for that.
2436  */
2437 static void
2439  uint32 targettaghash,
2440  SERIALIZABLEXACT *sxact)
2441 {
2442  PREDICATELOCKTARGET *target;
2443  PREDICATELOCKTAG locktag;
2444  PREDICATELOCK *lock;
2445  LWLock *partitionLock;
2446  bool found;
2447 
2448  partitionLock = PredicateLockHashPartitionLock(targettaghash);
2449 
2450  LWLockAcquire(SerializablePredicateListLock, LW_SHARED);
2451  if (IsInParallelMode())
2453  LWLockAcquire(partitionLock, LW_EXCLUSIVE);
2454 
2455  /* Make sure that the target is represented. */
2456  target = (PREDICATELOCKTARGET *)
2458  targettag, targettaghash,
2459  HASH_ENTER_NULL, &found);
2460  if (!target)
2461  ereport(ERROR,
2462  (errcode(ERRCODE_OUT_OF_MEMORY),
2463  errmsg("out of shared memory"),
2464  errhint("You might need to increase %s.", "max_pred_locks_per_transaction")));
2465  if (!found)
2466  dlist_init(&target->predicateLocks);
2467 
2468  /* We've got the sxact and target, make sure they're joined. */
2469  locktag.myTarget = target;
2470  locktag.myXact = sxact;
2471  lock = (PREDICATELOCK *)
2473  PredicateLockHashCodeFromTargetHashCode(&locktag, targettaghash),
2474  HASH_ENTER_NULL, &found);
2475  if (!lock)
2476  ereport(ERROR,
2477  (errcode(ERRCODE_OUT_OF_MEMORY),
2478  errmsg("out of shared memory"),
2479  errhint("You might need to increase %s.", "max_pred_locks_per_transaction")));
2480 
2481  if (!found)
2482  {
2483  dlist_push_tail(&target->predicateLocks, &lock->targetLink);
2484  dlist_push_tail(&sxact->predicateLocks, &lock->xactLink);
2486  }
2487 
2488  LWLockRelease(partitionLock);
2489  if (IsInParallelMode())
2491  LWLockRelease(SerializablePredicateListLock);
2492 }
2493 
2494 /*
2495  * Acquire a predicate lock on the specified target for the current
2496  * connection if not already held. This updates the local lock table
2497  * and uses it to implement granularity promotion. It will consolidate
2498  * multiple locks into a coarser lock if warranted, and will release
2499  * any finer-grained locks covered by the new one.
2500  */
2501 static void
2503 {
2504  uint32 targettaghash;
2505  bool found;
2506  LOCALPREDICATELOCK *locallock;
2507 
2508  /* Do we have the lock already, or a covering lock? */
2509  if (PredicateLockExists(targettag))
2510  return;
2511 
2512  if (CoarserLockCovers(targettag))
2513  return;
2514 
2515  /* the same hash and LW lock apply to the lock target and the local lock. */
2516  targettaghash = PredicateLockTargetTagHashCode(targettag);
2517 
2518  /* Acquire lock in local table */
2519  locallock = (LOCALPREDICATELOCK *)
2521  targettag, targettaghash,
2522  HASH_ENTER, &found);
2523  locallock->held = true;
2524  if (!found)
2525  locallock->childLocks = 0;
2526 
2527  /* Actually create the lock */
2528  CreatePredicateLock(targettag, targettaghash, MySerializableXact);
2529 
2530  /*
2531  * Lock has been acquired. Check whether it should be promoted to a
2532  * coarser granularity, or whether there are finer-granularity locks to
2533  * clean up.
2534  */
2535  if (CheckAndPromotePredicateLockRequest(targettag))
2536  {
2537  /*
2538  * Lock request was promoted to a coarser-granularity lock, and that
2539  * lock was acquired. It will delete this lock and any of its
2540  * children, so we're done.
2541  */
2542  }
2543  else
2544  {
2545  /* Clean up any finer-granularity locks */
2547  DeleteChildTargetLocks(targettag);
2548  }
2549 }
2550 
2551 
2552 /*
2553  * PredicateLockRelation
2554  *
2555  * Gets a predicate lock at the relation level.
2556  * Skip if not in full serializable transaction isolation level.
2557  * Skip if this is a temporary table.
2558  * Clear any finer-grained predicate locks this session has on the relation.
2559  */
2560 void
2562 {
2564 
2565  if (!SerializationNeededForRead(relation, snapshot))
2566  return;
2567 
2569  relation->rd_locator.dbOid,
2570  relation->rd_id);
2571  PredicateLockAcquire(&tag);
2572 }
2573 
2574 /*
2575  * PredicateLockPage
2576  *
2577  * Gets a predicate lock at the page level.
2578  * Skip if not in full serializable transaction isolation level.
2579  * Skip if this is a temporary table.
2580  * Skip if a coarser predicate lock already covers this page.
2581  * Clear any finer-grained predicate locks this session has on the relation.
2582  */
2583 void
2585 {
2587 
2588  if (!SerializationNeededForRead(relation, snapshot))
2589  return;
2590 
2592  relation->rd_locator.dbOid,
2593  relation->rd_id,
2594  blkno);
2595  PredicateLockAcquire(&tag);
2596 }
2597 
2598 /*
2599  * PredicateLockTID
2600  *
2601  * Gets a predicate lock at the tuple level.
2602  * Skip if not in full serializable transaction isolation level.
2603  * Skip if this is a temporary table.
2604  */
2605 void
2607  TransactionId tuple_xid)
2608 {
2610 
2611  if (!SerializationNeededForRead(relation, snapshot))
2612  return;
2613 
2614  /*
2615  * Return if this xact wrote it.
2616  */
2617  if (relation->rd_index == NULL)
2618  {
2619  /* If we wrote it; we already have a write lock. */
2620  if (TransactionIdIsCurrentTransactionId(tuple_xid))
2621  return;
2622  }
2623 
2624  /*
2625  * Do quick-but-not-definitive test for a relation lock first. This will
2626  * never cause a return when the relation is *not* locked, but will
2627  * occasionally let the check continue when there really *is* a relation
2628  * level lock.
2629  */
2631  relation->rd_locator.dbOid,
2632  relation->rd_id);
2633  if (PredicateLockExists(&tag))
2634  return;
2635 
2637  relation->rd_locator.dbOid,
2638  relation->rd_id,
2641  PredicateLockAcquire(&tag);
2642 }
2643 
2644 
2645 /*
2646  * DeleteLockTarget
2647  *
2648  * Remove a predicate lock target along with any locks held for it.
2649  *
2650  * Caller must hold SerializablePredicateListLock and the
2651  * appropriate hash partition lock for the target.
2652  */
2653 static void
2655 {
2656  dlist_mutable_iter iter;
2657 
2658  Assert(LWLockHeldByMeInMode(SerializablePredicateListLock,
2659  LW_EXCLUSIVE));
2661 
2662  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
2663 
2664  dlist_foreach_modify(iter, &target->predicateLocks)
2665  {
2666  PREDICATELOCK *predlock =
2667  dlist_container(PREDICATELOCK, targetLink, iter.cur);
2668  bool found;
2669 
2670  dlist_delete(&(predlock->xactLink));
2671  dlist_delete(&(predlock->targetLink));
2672 
2675  &predlock->tag,
2677  targettaghash),
2678  HASH_REMOVE, &found);
2679  Assert(found);
2680  }
2681  LWLockRelease(SerializableXactHashLock);
2682 
2683  /* Remove the target itself, if possible. */
2684  RemoveTargetIfNoLongerUsed(target, targettaghash);
2685 }
2686 
2687 
2688 /*
2689  * TransferPredicateLocksToNewTarget
2690  *
2691  * Move or copy all the predicate locks for a lock target, for use by
2692  * index page splits/combines and other things that create or replace
2693  * lock targets. If 'removeOld' is true, the old locks and the target
2694  * will be removed.
2695  *
2696  * Returns true on success, or false if we ran out of shared memory to
2697  * allocate the new target or locks. Guaranteed to always succeed if
2698  * removeOld is set (by using the scratch entry in PredicateLockTargetHash
2699  * for scratch space).
2700  *
2701  * Warning: the "removeOld" option should be used only with care,
2702  * because this function does not (indeed, can not) update other
2703  * backends' LocalPredicateLockHash. If we are only adding new
2704  * entries, this is not a problem: the local lock table is used only
2705  * as a hint, so missing entries for locks that are held are
2706  * OK. Having entries for locks that are no longer held, as can happen
2707  * when using "removeOld", is not in general OK. We can only use it
2708  * safely when replacing a lock with a coarser-granularity lock that
2709  * covers it, or if we are absolutely certain that no one will need to
2710  * refer to that lock in the future.
2711  *
2712  * Caller must hold SerializablePredicateListLock exclusively.
2713  */
2714 static bool
2716  PREDICATELOCKTARGETTAG newtargettag,
2717  bool removeOld)
2718 {
2719  uint32 oldtargettaghash;
2720  LWLock *oldpartitionLock;
2721  PREDICATELOCKTARGET *oldtarget;
2722  uint32 newtargettaghash;
2723  LWLock *newpartitionLock;
2724  bool found;
2725  bool outOfShmem = false;
2726 
2727  Assert(LWLockHeldByMeInMode(SerializablePredicateListLock,
2728  LW_EXCLUSIVE));
2729 
2730  oldtargettaghash = PredicateLockTargetTagHashCode(&oldtargettag);
2731  newtargettaghash = PredicateLockTargetTagHashCode(&newtargettag);
2732  oldpartitionLock = PredicateLockHashPartitionLock(oldtargettaghash);
2733  newpartitionLock = PredicateLockHashPartitionLock(newtargettaghash);
2734 
2735  if (removeOld)
2736  {
2737  /*
2738  * Remove the dummy entry to give us scratch space, so we know we'll
2739  * be able to create the new lock target.
2740  */
2741  RemoveScratchTarget(false);
2742  }
2743 
2744  /*
2745  * We must get the partition locks in ascending sequence to avoid
2746  * deadlocks. If old and new partitions are the same, we must request the
2747  * lock only once.
2748  */
2749  if (oldpartitionLock < newpartitionLock)
2750  {
2751  LWLockAcquire(oldpartitionLock,
2752  (removeOld ? LW_EXCLUSIVE : LW_SHARED));
2753  LWLockAcquire(newpartitionLock, LW_EXCLUSIVE);
2754  }
2755  else if (oldpartitionLock > newpartitionLock)
2756  {
2757  LWLockAcquire(newpartitionLock, LW_EXCLUSIVE);
2758  LWLockAcquire(oldpartitionLock,
2759  (removeOld ? LW_EXCLUSIVE : LW_SHARED));
2760  }
2761  else
2762  LWLockAcquire(newpartitionLock, LW_EXCLUSIVE);
2763 
2764  /*
2765  * Look for the old target. If not found, that's OK; no predicate locks
2766  * are affected, so we can just clean up and return. If it does exist,
2767  * walk its list of predicate locks and move or copy them to the new
2768  * target.
2769  */
2771  &oldtargettag,
2772  oldtargettaghash,
2773  HASH_FIND, NULL);
2774 
2775  if (oldtarget)
2776  {
2777  PREDICATELOCKTARGET *newtarget;
2778  PREDICATELOCKTAG newpredlocktag;
2779  dlist_mutable_iter iter;
2780 
2782  &newtargettag,
2783  newtargettaghash,
2784  HASH_ENTER_NULL, &found);
2785 
2786  if (!newtarget)
2787  {
2788  /* Failed to allocate due to insufficient shmem */
2789  outOfShmem = true;
2790  goto exit;
2791  }
2792 
2793  /* If we created a new entry, initialize it */
2794  if (!found)
2795  dlist_init(&newtarget->predicateLocks);
2796 
2797  newpredlocktag.myTarget = newtarget;
2798 
2799  /*
2800  * Loop through all the locks on the old target, replacing them with
2801  * locks on the new target.
2802  */
2803  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
2804 
2805  dlist_foreach_modify(iter, &oldtarget->predicateLocks)
2806  {
2807  PREDICATELOCK *oldpredlock =
2808  dlist_container(PREDICATELOCK, targetLink, iter.cur);
2809  PREDICATELOCK *newpredlock;
2810  SerCommitSeqNo oldCommitSeqNo = oldpredlock->commitSeqNo;
2811 
2812  newpredlocktag.myXact = oldpredlock->tag.myXact;
2813 
2814  if (removeOld)
2815  {
2816  dlist_delete(&(oldpredlock->xactLink));
2817  dlist_delete(&(oldpredlock->targetLink));
2818 
2821  &oldpredlock->tag,
2823  oldtargettaghash),
2824  HASH_REMOVE, &found);
2825  Assert(found);
2826  }
2827 
2828  newpredlock = (PREDICATELOCK *)
2830  &newpredlocktag,
2832  newtargettaghash),
2834  &found);
2835  if (!newpredlock)
2836  {
2837  /* Out of shared memory. Undo what we've done so far. */
2838  LWLockRelease(SerializableXactHashLock);
2839  DeleteLockTarget(newtarget, newtargettaghash);
2840  outOfShmem = true;
2841  goto exit;
2842  }
2843  if (!found)
2844  {
2845  dlist_push_tail(&(newtarget->predicateLocks),
2846  &(newpredlock->targetLink));
2847  dlist_push_tail(&(newpredlocktag.myXact->predicateLocks),
2848  &(newpredlock->xactLink));
2849  newpredlock->commitSeqNo = oldCommitSeqNo;
2850  }
2851  else
2852  {
2853  if (newpredlock->commitSeqNo < oldCommitSeqNo)
2854  newpredlock->commitSeqNo = oldCommitSeqNo;
2855  }
2856 
2857  Assert(newpredlock->commitSeqNo != 0);
2858  Assert((newpredlock->commitSeqNo == InvalidSerCommitSeqNo)
2859  || (newpredlock->tag.myXact == OldCommittedSxact));
2860  }
2861  LWLockRelease(SerializableXactHashLock);
2862 
2863  if (removeOld)
2864  {
2865  Assert(dlist_is_empty(&oldtarget->predicateLocks));
2866  RemoveTargetIfNoLongerUsed(oldtarget, oldtargettaghash);
2867  }
2868  }
2869 
2870 
2871 exit:
2872  /* Release partition locks in reverse order of acquisition. */
2873  if (oldpartitionLock < newpartitionLock)
2874  {
2875  LWLockRelease(newpartitionLock);
2876  LWLockRelease(oldpartitionLock);
2877  }
2878  else if (oldpartitionLock > newpartitionLock)
2879  {
2880  LWLockRelease(oldpartitionLock);
2881  LWLockRelease(newpartitionLock);
2882  }
2883  else
2884  LWLockRelease(newpartitionLock);
2885 
2886  if (removeOld)
2887  {
2888  /* We shouldn't run out of memory if we're moving locks */
2889  Assert(!outOfShmem);
2890 
2891  /* Put the scratch entry back */
2892  RestoreScratchTarget(false);
2893  }
2894 
2895  return !outOfShmem;
2896 }
2897 
2898 /*
2899  * Drop all predicate locks of any granularity from the specified relation,
2900  * which can be a heap relation or an index relation. If 'transfer' is true,
2901  * acquire a relation lock on the heap for any transactions with any lock(s)
2902  * on the specified relation.
2903  *
2904  * This requires grabbing a lot of LW locks and scanning the entire lock
2905  * target table for matches. That makes this more expensive than most
2906  * predicate lock management functions, but it will only be called for DDL
2907  * type commands that are expensive anyway, and there are fast returns when
2908  * no serializable transactions are active or the relation is temporary.
2909  *
2910  * We don't use the TransferPredicateLocksToNewTarget function because it
2911  * acquires its own locks on the partitions of the two targets involved,
2912  * and we'll already be holding all partition locks.
2913  *
2914  * We can't throw an error from here, because the call could be from a
2915  * transaction which is not serializable.
2916  *
2917  * NOTE: This is currently only called with transfer set to true, but that may
2918  * change. If we decide to clean up the locks from a table on commit of a
2919  * transaction which executed DROP TABLE, the false condition will be useful.
2920  */
2921 static void
2923 {
2924  HASH_SEQ_STATUS seqstat;
2925  PREDICATELOCKTARGET *oldtarget;
2926  PREDICATELOCKTARGET *heaptarget;
2927  Oid dbId;
2928  Oid relId;
2929  Oid heapId;
2930  int i;
2931  bool isIndex;
2932  bool found;
2933  uint32 heaptargettaghash;
2934 
2935  /*
2936  * Bail out quickly if there are no serializable transactions running.
2937  * It's safe to check this without taking locks because the caller is
2938  * holding an ACCESS EXCLUSIVE lock on the relation. No new locks which
2939  * would matter here can be acquired while that is held.
2940  */
2942  return;
2943 
2944  if (!PredicateLockingNeededForRelation(relation))
2945  return;
2946 
2947  dbId = relation->rd_locator.dbOid;
2948  relId = relation->rd_id;
2949  if (relation->rd_index == NULL)
2950  {
2951  isIndex = false;
2952  heapId = relId;
2953  }
2954  else
2955  {
2956  isIndex = true;
2957  heapId = relation->rd_index->indrelid;
2958  }
2959  Assert(heapId != InvalidOid);
2960  Assert(transfer || !isIndex); /* index OID only makes sense with
2961  * transfer */
2962 
2963  /* Retrieve first time needed, then keep. */
2964  heaptargettaghash = 0;
2965  heaptarget = NULL;
2966 
2967  /* Acquire locks on all lock partitions */
2968  LWLockAcquire(SerializablePredicateListLock, LW_EXCLUSIVE);
2969  for (i = 0; i < NUM_PREDICATELOCK_PARTITIONS; i++)
2971  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
2972 
2973  /*
2974  * Remove the dummy entry to give us scratch space, so we know we'll be
2975  * able to create the new lock target.
2976  */
2977  if (transfer)
2978  RemoveScratchTarget(true);
2979 
2980  /* Scan through target map */
2982 
2983  while ((oldtarget = (PREDICATELOCKTARGET *) hash_seq_search(&seqstat)))
2984  {
2985  dlist_mutable_iter iter;
2986 
2987  /*
2988  * Check whether this is a target which needs attention.
2989  */
2990  if (GET_PREDICATELOCKTARGETTAG_RELATION(oldtarget->tag) != relId)
2991  continue; /* wrong relation id */
2992  if (GET_PREDICATELOCKTARGETTAG_DB(oldtarget->tag) != dbId)
2993  continue; /* wrong database id */
2994  if (transfer && !isIndex
2996  continue; /* already the right lock */
2997 
2998  /*
2999  * If we made it here, we have work to do. We make sure the heap
3000  * relation lock exists, then we walk the list of predicate locks for
3001  * the old target we found, moving all locks to the heap relation lock
3002  * -- unless they already hold that.
3003  */
3004 
3005  /*
3006  * First make sure we have the heap relation target. We only need to
3007  * do this once.
3008  */
3009  if (transfer && heaptarget == NULL)
3010  {
3011  PREDICATELOCKTARGETTAG heaptargettag;
3012 
3013  SET_PREDICATELOCKTARGETTAG_RELATION(heaptargettag, dbId, heapId);
3014  heaptargettaghash = PredicateLockTargetTagHashCode(&heaptargettag);
3016  &heaptargettag,
3017  heaptargettaghash,
3018  HASH_ENTER, &found);
3019  if (!found)
3020  dlist_init(&heaptarget->predicateLocks);
3021  }
3022 
3023  /*
3024  * Loop through all the locks on the old target, replacing them with
3025  * locks on the new target.
3026  */
3027  dlist_foreach_modify(iter, &oldtarget->predicateLocks)
3028  {
3029  PREDICATELOCK *oldpredlock =
3030  dlist_container(PREDICATELOCK, targetLink, iter.cur);
3031  PREDICATELOCK *newpredlock;
3032  SerCommitSeqNo oldCommitSeqNo;
3033  SERIALIZABLEXACT *oldXact;
3034 
3035  /*
3036  * Remove the old lock first. This avoids the chance of running
3037  * out of lock structure entries for the hash table.
3038  */
3039  oldCommitSeqNo = oldpredlock->commitSeqNo;
3040  oldXact = oldpredlock->tag.myXact;
3041 
3042  dlist_delete(&(oldpredlock->xactLink));
3043 
3044  /*
3045  * No need for retail delete from oldtarget list, we're removing
3046  * the whole target anyway.
3047  */
3049  &oldpredlock->tag,
3050  HASH_REMOVE, &found);
3051  Assert(found);
3052 
3053  if (transfer)
3054  {
3055  PREDICATELOCKTAG newpredlocktag;
3056 
3057  newpredlocktag.myTarget = heaptarget;
3058  newpredlocktag.myXact = oldXact;
3059  newpredlock = (PREDICATELOCK *)
3061  &newpredlocktag,
3063  heaptargettaghash),
3064  HASH_ENTER,
3065  &found);
3066  if (!found)
3067  {
3068  dlist_push_tail(&(heaptarget->predicateLocks),
3069  &(newpredlock->targetLink));
3070  dlist_push_tail(&(newpredlocktag.myXact->predicateLocks),
3071  &(newpredlock->xactLink));
3072  newpredlock->commitSeqNo = oldCommitSeqNo;
3073  }
3074  else
3075  {
3076  if (newpredlock->commitSeqNo < oldCommitSeqNo)
3077  newpredlock->commitSeqNo = oldCommitSeqNo;
3078  }
3079 
3080  Assert(newpredlock->commitSeqNo != 0);
3081  Assert((newpredlock->commitSeqNo == InvalidSerCommitSeqNo)
3082  || (newpredlock->tag.myXact == OldCommittedSxact));
3083  }
3084  }
3085 
3087  &found);
3088  Assert(found);
3089  }
3090 
3091  /* Put the scratch entry back */
3092  if (transfer)
3093  RestoreScratchTarget(true);
3094 
3095  /* Release locks in reverse order */
3096  LWLockRelease(SerializableXactHashLock);
3097  for (i = NUM_PREDICATELOCK_PARTITIONS - 1; i >= 0; i--)
3099  LWLockRelease(SerializablePredicateListLock);
3100 }
3101 
3102 /*
3103  * TransferPredicateLocksToHeapRelation
3104  * For all transactions, transfer all predicate locks for the given
3105  * relation to a single relation lock on the heap.
3106  */
3107 void
3109 {
3110  DropAllPredicateLocksFromTable(relation, true);
3111 }
3112 
3113 
3114 /*
3115  * PredicateLockPageSplit
3116  *
3117  * Copies any predicate locks for the old page to the new page.
3118  * Skip if this is a temporary table or toast table.
3119  *
3120  * NOTE: A page split (or overflow) affects all serializable transactions,
3121  * even if it occurs in the context of another transaction isolation level.
3122  *
3123  * NOTE: This currently leaves the local copy of the locks without
3124  * information on the new lock which is in shared memory. This could cause
3125  * problems if enough page splits occur on locked pages without the processes
3126  * which hold the locks getting in and noticing.
3127  */
3128 void
3130  BlockNumber newblkno)
3131 {
3132  PREDICATELOCKTARGETTAG oldtargettag;
3133  PREDICATELOCKTARGETTAG newtargettag;
3134  bool success;
3135 
3136  /*
3137  * Bail out quickly if there are no serializable transactions running.
3138  *
3139  * It's safe to do this check without taking any additional locks. Even if
3140  * a serializable transaction starts concurrently, we know it can't take
3141  * any SIREAD locks on the page being split because the caller is holding
3142  * the associated buffer page lock. Memory reordering isn't an issue; the
3143  * memory barrier in the LWLock acquisition guarantees that this read
3144  * occurs while the buffer page lock is held.
3145  */
3147  return;
3148 
3149  if (!PredicateLockingNeededForRelation(relation))
3150  return;
3151 
3152  Assert(oldblkno != newblkno);
3153  Assert(BlockNumberIsValid(oldblkno));
3154  Assert(BlockNumberIsValid(newblkno));
3155 
3156  SET_PREDICATELOCKTARGETTAG_PAGE(oldtargettag,
3157  relation->rd_locator.dbOid,
3158  relation->rd_id,
3159  oldblkno);
3160  SET_PREDICATELOCKTARGETTAG_PAGE(newtargettag,
3161  relation->rd_locator.dbOid,
3162  relation->rd_id,
3163  newblkno);
3164 
3165  LWLockAcquire(SerializablePredicateListLock, LW_EXCLUSIVE);
3166 
3167  /*
3168  * Try copying the locks over to the new page's tag, creating it if
3169  * necessary.
3170  */
3172  newtargettag,
3173  false);
3174 
3175  if (!success)
3176  {
3177  /*
3178  * No more predicate lock entries are available. Failure isn't an
3179  * option here, so promote the page lock to a relation lock.
3180  */
3181 
3182  /* Get the parent relation lock's lock tag */
3183  success = GetParentPredicateLockTag(&oldtargettag,
3184  &newtargettag);
3185  Assert(success);
3186 
3187  /*
3188  * Move the locks to the parent. This shouldn't fail.
3189  *
3190  * Note that here we are removing locks held by other backends,
3191  * leading to a possible inconsistency in their local lock hash table.
3192  * This is OK because we're replacing it with a lock that covers the
3193  * old one.
3194  */
3196  newtargettag,
3197  true);
3198  Assert(success);
3199  }
3200 
3201  LWLockRelease(SerializablePredicateListLock);
3202 }
3203 
3204 /*
3205  * PredicateLockPageCombine
3206  *
3207  * Combines predicate locks for two existing pages.
3208  * Skip if this is a temporary table or toast table.
3209  *
3210  * NOTE: A page combine affects all serializable transactions, even if it
3211  * occurs in the context of another transaction isolation level.
3212  */
3213 void
3215  BlockNumber newblkno)
3216 {
3217  /*
3218  * Page combines differ from page splits in that we ought to be able to
3219  * remove the locks on the old page after transferring them to the new
3220  * page, instead of duplicating them. However, because we can't edit other
3221  * backends' local lock tables, removing the old lock would leave them
3222  * with an entry in their LocalPredicateLockHash for a lock they're not
3223  * holding, which isn't acceptable. So we wind up having to do the same
3224  * work as a page split, acquiring a lock on the new page and keeping the
3225  * old page locked too. That can lead to some false positives, but should
3226  * be rare in practice.
3227  */
3228  PredicateLockPageSplit(relation, oldblkno, newblkno);
3229 }
3230 
3231 /*
3232  * Walk the list of in-progress serializable transactions and find the new
3233  * xmin.
3234  */
3235 static void
3237 {
3238  dlist_iter iter;
3239 
3240  Assert(LWLockHeldByMe(SerializableXactHashLock));
3241 
3244 
3246  {
3247  SERIALIZABLEXACT *sxact =
3248  dlist_container(SERIALIZABLEXACT, xactLink, iter.cur);
3249 
3250  if (!SxactIsRolledBack(sxact)
3251  && !SxactIsCommitted(sxact)
3252  && sxact != OldCommittedSxact)
3253  {
3254  Assert(sxact->xmin != InvalidTransactionId);
3256  || TransactionIdPrecedes(sxact->xmin,
3258  {
3259  PredXact->SxactGlobalXmin = sxact->xmin;
3261  }
3262  else if (TransactionIdEquals(sxact->xmin,
3265  }
3266  }
3267 
3269 }
3270 
3271 /*
3272  * ReleasePredicateLocks
3273  *
3274  * Releases predicate locks based on completion of the current transaction,
3275  * whether committed or rolled back. It can also be called for a read only
3276  * transaction when it becomes impossible for the transaction to become
3277  * part of a dangerous structure.
3278  *
3279  * We do nothing unless this is a serializable transaction.
3280  *
3281  * This method must ensure that shared memory hash tables are cleaned
3282  * up in some relatively timely fashion.
3283  *
3284  * If this transaction is committing and is holding any predicate locks,
3285  * it must be added to a list of completed serializable transactions still
3286  * holding locks.
3287  *
3288  * If isReadOnlySafe is true, then predicate locks are being released before
3289  * the end of the transaction because MySerializableXact has been determined
3290  * to be RO_SAFE. In non-parallel mode we can release it completely, but it
3291  * in parallel mode we partially release the SERIALIZABLEXACT and keep it
3292  * around until the end of the transaction, allowing each backend to clear its
3293  * MySerializableXact variable and benefit from the optimization in its own
3294  * time.
3295  */
3296 void
3297 ReleasePredicateLocks(bool isCommit, bool isReadOnlySafe)
3298 {
3299  bool partiallyReleasing = false;
3300  bool needToClear;
3301  SERIALIZABLEXACT *roXact;
3302  dlist_mutable_iter iter;
3303 
3304  /*
3305  * We can't trust XactReadOnly here, because a transaction which started
3306  * as READ WRITE can show as READ ONLY later, e.g., within
3307  * subtransactions. We want to flag a transaction as READ ONLY if it
3308  * commits without writing so that de facto READ ONLY transactions get the
3309  * benefit of some RO optimizations, so we will use this local variable to
3310  * get some cleanup logic right which is based on whether the transaction
3311  * was declared READ ONLY at the top level.
3312  */
3313  bool topLevelIsDeclaredReadOnly;
3314 
3315  /* We can't be both committing and releasing early due to RO_SAFE. */
3316  Assert(!(isCommit && isReadOnlySafe));
3317 
3318  /* Are we at the end of a transaction, that is, a commit or abort? */
3319  if (!isReadOnlySafe)
3320  {
3321  /*
3322  * Parallel workers mustn't release predicate locks at the end of
3323  * their transaction. The leader will do that at the end of its
3324  * transaction.
3325  */
3326  if (IsParallelWorker())
3327  {
3329  return;
3330  }
3331 
3332  /*
3333  * By the time the leader in a parallel query reaches end of
3334  * transaction, it has waited for all workers to exit.
3335  */
3337 
3338  /*
3339  * If the leader in a parallel query earlier stashed a partially
3340  * released SERIALIZABLEXACT for final clean-up at end of transaction
3341  * (because workers might still have been accessing it), then it's
3342  * time to restore it.
3343  */
3345  {
3350  }
3351  }
3352 
3354  {
3355  Assert(LocalPredicateLockHash == NULL);
3356  return;
3357  }
3358 
3359  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
3360 
3361  /*
3362  * If the transaction is committing, but it has been partially released
3363  * already, then treat this as a roll back. It was marked as rolled back.
3364  */
3366  isCommit = false;
3367 
3368  /*
3369  * If we're called in the middle of a transaction because we discovered
3370  * that the SXACT_FLAG_RO_SAFE flag was set, then we'll partially release
3371  * it (that is, release the predicate locks and conflicts, but not the
3372  * SERIALIZABLEXACT itself) if we're the first backend to have noticed.
3373  */
3374  if (isReadOnlySafe && IsInParallelMode())
3375  {
3376  /*
3377  * The leader needs to stash a pointer to it, so that it can
3378  * completely release it at end-of-transaction.
3379  */
3380  if (!IsParallelWorker())
3382 
3383  /*
3384  * The first backend to reach this condition will partially release
3385  * the SERIALIZABLEXACT. All others will just clear their
3386  * backend-local state so that they stop doing SSI checks for the rest
3387  * of the transaction.
3388  */
3390  {
3391  LWLockRelease(SerializableXactHashLock);
3393  return;
3394  }
3395  else
3396  {
3398  partiallyReleasing = true;
3399  /* ... and proceed to perform the partial release below. */
3400  }
3401  }
3402  Assert(!isCommit || SxactIsPrepared(MySerializableXact));
3403  Assert(!isCommit || !SxactIsDoomed(MySerializableXact));
3407 
3408  /* may not be serializable during COMMIT/ROLLBACK PREPARED */
3410 
3411  /* We'd better not already be on the cleanup list. */
3413 
3414  topLevelIsDeclaredReadOnly = SxactIsReadOnly(MySerializableXact);
3415 
3416  /*
3417  * We don't hold XidGenLock lock here, assuming that TransactionId is
3418  * atomic!
3419  *
3420  * If this value is changing, we don't care that much whether we get the
3421  * old or new value -- it is just used to determine how far
3422  * SxactGlobalXmin must advance before this transaction can be fully
3423  * cleaned up. The worst that could happen is we wait for one more
3424  * transaction to complete before freeing some RAM; correctness of visible
3425  * behavior is not affected.
3426  */
3428 
3429  /*
3430  * If it's not a commit it's either a rollback or a read-only transaction
3431  * flagged SXACT_FLAG_RO_SAFE, and we can clear our locks immediately.
3432  */
3433  if (isCommit)
3434  {
3437  /* Recognize implicit read-only transaction (commit without write). */
3438  if (!MyXactDidWrite)
3440  }
3441  else
3442  {
3443  /*
3444  * The DOOMED flag indicates that we intend to roll back this
3445  * transaction and so it should not cause serialization failures for
3446  * other transactions that conflict with it. Note that this flag might
3447  * already be set, if another backend marked this transaction for
3448  * abort.
3449  *
3450  * The ROLLED_BACK flag further indicates that ReleasePredicateLocks
3451  * has been called, and so the SerializableXact is eligible for
3452  * cleanup. This means it should not be considered when calculating
3453  * SxactGlobalXmin.
3454  */
3457 
3458  /*
3459  * If the transaction was previously prepared, but is now failing due
3460  * to a ROLLBACK PREPARED or (hopefully very rare) error after the
3461  * prepare, clear the prepared flag. This simplifies conflict
3462  * checking.
3463  */
3465  }
3466 
3467  if (!topLevelIsDeclaredReadOnly)
3468  {
3470  if (--(PredXact->WritableSxactCount) == 0)
3471  {
3472  /*
3473  * Release predicate locks and rw-conflicts in for all committed
3474  * transactions. There are no longer any transactions which might
3475  * conflict with the locks and no chance for new transactions to
3476  * overlap. Similarly, existing conflicts in can't cause pivots,
3477  * and any conflicts in which could have completed a dangerous
3478  * structure would already have caused a rollback, so any
3479  * remaining ones must be benign.
3480  */
3482  }
3483  }
3484  else
3485  {
3486  /*
3487  * Read-only transactions: clear the list of transactions that might
3488  * make us unsafe. Note that we use 'inLink' for the iteration as
3489  * opposed to 'outLink' for the r/w xacts.
3490  */
3492  {
3493  RWConflict possibleUnsafeConflict =
3494  dlist_container(RWConflictData, inLink, iter.cur);
3495 
3496  Assert(!SxactIsReadOnly(possibleUnsafeConflict->sxactOut));
3497  Assert(MySerializableXact == possibleUnsafeConflict->sxactIn);
3498 
3499  ReleaseRWConflict(possibleUnsafeConflict);
3500  }
3501  }
3502 
3503  /* Check for conflict out to old committed transactions. */
3504  if (isCommit
3507  {
3508  /*
3509  * we don't know which old committed transaction we conflicted with,
3510  * so be conservative and use FirstNormalSerCommitSeqNo here
3511  */
3515  }
3516 
3517  /*
3518  * Release all outConflicts to committed transactions. If we're rolling
3519  * back clear them all. Set SXACT_FLAG_CONFLICT_OUT if any point to
3520  * previously committed transactions.
3521  */
3523  {
3524  RWConflict conflict =
3525  dlist_container(RWConflictData, outLink, iter.cur);
3526 
3527  if (isCommit
3529  && SxactIsCommitted(conflict->sxactIn))
3530  {
3535  }
3536 
3537  if (!isCommit
3538  || SxactIsCommitted(conflict->sxactIn)
3540  ReleaseRWConflict(conflict);
3541  }
3542 
3543  /*
3544  * Release all inConflicts from committed and read-only transactions. If
3545  * we're rolling back, clear them all.
3546  */
3548  {
3549  RWConflict conflict =
3550  dlist_container(RWConflictData, inLink, iter.cur);
3551 
3552  if (!isCommit
3553  || SxactIsCommitted(conflict->sxactOut)
3554  || SxactIsReadOnly(conflict->sxactOut))
3555  ReleaseRWConflict(conflict);
3556  }
3557 
3558  if (!topLevelIsDeclaredReadOnly)
3559  {
3560  /*
3561  * Remove ourselves from the list of possible conflicts for concurrent
3562  * READ ONLY transactions, flagging them as unsafe if we have a
3563  * conflict out. If any are waiting DEFERRABLE transactions, wake them
3564  * up if they are known safe or known unsafe.
3565  */
3567  {
3568  RWConflict possibleUnsafeConflict =
3569  dlist_container(RWConflictData, outLink, iter.cur);
3570 
3571  roXact = possibleUnsafeConflict->sxactIn;
3572  Assert(MySerializableXact == possibleUnsafeConflict->sxactOut);
3573  Assert(SxactIsReadOnly(roXact));
3574 
3575  /* Mark conflicted if necessary. */
3576  if (isCommit
3577  && MyXactDidWrite
3580  <= roXact->SeqNo.lastCommitBeforeSnapshot))
3581  {
3582  /*
3583  * This releases possibleUnsafeConflict (as well as all other
3584  * possible conflicts for roXact)
3585  */
3586  FlagSxactUnsafe(roXact);
3587  }
3588  else
3589  {
3590  ReleaseRWConflict(possibleUnsafeConflict);
3591 
3592  /*
3593  * If we were the last possible conflict, flag it safe. The
3594  * transaction can now safely release its predicate locks (but
3595  * that transaction's backend has to do that itself).
3596  */
3598  roXact->flags |= SXACT_FLAG_RO_SAFE;
3599  }
3600 
3601  /*
3602  * Wake up the process for a waiting DEFERRABLE transaction if we
3603  * now know it's either safe or conflicted.
3604  */
3605  if (SxactIsDeferrableWaiting(roXact) &&
3606  (SxactIsROUnsafe(roXact) || SxactIsROSafe(roXact)))
3607  ProcSendSignal(roXact->pgprocno);
3608  }
3609  }
3610 
3611  /*
3612  * Check whether it's time to clean up old transactions. This can only be
3613  * done when the last serializable transaction with the oldest xmin among
3614  * serializable transactions completes. We then find the "new oldest"
3615  * xmin and purge any transactions which finished before this transaction
3616  * was launched.
3617  *
3618  * For parallel queries in read-only transactions, it might run twice. We
3619  * only release the reference on the first call.
3620  */
3621  needToClear = false;
3622  if ((partiallyReleasing ||
3626  {
3628  if (--(PredXact->SxactGlobalXminCount) == 0)
3629  {
3631  needToClear = true;
3632  }
3633  }
3634 
3635  LWLockRelease(SerializableXactHashLock);
3636 
3637  LWLockAcquire(SerializableFinishedListLock, LW_EXCLUSIVE);
3638 
3639  /* Add this to the list of transactions to check for later cleanup. */
3640  if (isCommit)
3643 
3644  /*
3645  * If we're releasing a RO_SAFE transaction in parallel mode, we'll only
3646  * partially release it. That's necessary because other backends may have
3647  * a reference to it. The leader will release the SERIALIZABLEXACT itself
3648  * at the end of the transaction after workers have stopped running.
3649  */
3650  if (!isCommit)
3652  isReadOnlySafe && IsInParallelMode(),
3653  false);
3654 
3655  LWLockRelease(SerializableFinishedListLock);
3656 
3657  if (needToClear)
3659 
3661 }
3662 
3663 static void
3665 {
3667  MyXactDidWrite = false;
3668 
3669  /* Delete per-transaction lock table */
3670  if (LocalPredicateLockHash != NULL)
3671  {
3673  LocalPredicateLockHash = NULL;
3674  }
3675 }
3676 
3677 /*
3678  * Clear old predicate locks, belonging to committed transactions that are no
3679  * longer interesting to any in-progress transaction.
3680  */
3681 static void
3683 {
3684  dlist_mutable_iter iter;
3685 
3686  /*
3687  * Loop through finished transactions. They are in commit order, so we can
3688  * stop as soon as we find one that's still interesting.
3689  */
3690  LWLockAcquire(SerializableFinishedListLock, LW_EXCLUSIVE);
3691  LWLockAcquire(SerializableXactHashLock, LW_SHARED);
3693  {
3694  SERIALIZABLEXACT *finishedSxact =
3695  dlist_container(SERIALIZABLEXACT, finishedLink, iter.cur);
3696 
3700  {
3701  /*
3702  * This transaction committed before any in-progress transaction
3703  * took its snapshot. It's no longer interesting.
3704  */
3705  LWLockRelease(SerializableXactHashLock);
3706  dlist_delete_thoroughly(&finishedSxact->finishedLink);
3707  ReleaseOneSerializableXact(finishedSxact, false, false);
3708  LWLockAcquire(SerializableXactHashLock, LW_SHARED);
3709  }
3710  else if (finishedSxact->commitSeqNo > PredXact->HavePartialClearedThrough
3711  && finishedSxact->commitSeqNo <= PredXact->CanPartialClearThrough)
3712  {
3713  /*
3714  * Any active transactions that took their snapshot before this
3715  * transaction committed are read-only, so we can clear part of
3716  * its state.
3717  */
3718  LWLockRelease(SerializableXactHashLock);
3719 
3720  if (SxactIsReadOnly(finishedSxact))
3721  {
3722  /* A read-only transaction can be removed entirely */
3723  dlist_delete_thoroughly(&(finishedSxact->finishedLink));
3724  ReleaseOneSerializableXact(finishedSxact, false, false);
3725  }
3726  else
3727  {
3728  /*
3729  * A read-write transaction can only be partially cleared. We
3730  * need to keep the SERIALIZABLEXACT but can release the
3731  * SIREAD locks and conflicts in.
3732  */
3733  ReleaseOneSerializableXact(finishedSxact, true, false);
3734  }
3735 
3737  LWLockAcquire(SerializableXactHashLock, LW_SHARED);
3738  }
3739  else
3740  {
3741  /* Still interesting. */
3742  break;
3743  }
3744  }
3745  LWLockRelease(SerializableXactHashLock);
3746 
3747  /*
3748  * Loop through predicate locks on dummy transaction for summarized data.
3749  */
3750  LWLockAcquire(SerializablePredicateListLock, LW_SHARED);
3752  {
3753  PREDICATELOCK *predlock =
3754  dlist_container(PREDICATELOCK, xactLink, iter.cur);
3755  bool canDoPartialCleanup;
3756 
3757  LWLockAcquire(SerializableXactHashLock, LW_SHARED);
3758  Assert(predlock->commitSeqNo != 0);
3760  canDoPartialCleanup = (predlock->commitSeqNo <= PredXact->CanPartialClearThrough);
3761  LWLockRelease(SerializableXactHashLock);
3762 
3763  /*
3764  * If this lock originally belonged to an old enough transaction, we
3765  * can release it.
3766  */
3767  if (canDoPartialCleanup)
3768  {
3769  PREDICATELOCKTAG tag;
3770  PREDICATELOCKTARGET *target;
3771  PREDICATELOCKTARGETTAG targettag;
3772  uint32 targettaghash;
3773  LWLock *partitionLock;
3774 
3775  tag = predlock->tag;
3776  target = tag.myTarget;
3777  targettag = target->tag;
3778  targettaghash = PredicateLockTargetTagHashCode(&targettag);
3779  partitionLock = PredicateLockHashPartitionLock(targettaghash);
3780 
3781  LWLockAcquire(partitionLock, LW_EXCLUSIVE);
3782 
3783  dlist_delete(&(predlock->targetLink));
3784  dlist_delete(&(predlock->xactLink));
3785 
3788  targettaghash),
3789  HASH_REMOVE, NULL);
3790  RemoveTargetIfNoLongerUsed(target, targettaghash);
3791 
3792  LWLockRelease(partitionLock);
3793  }
3794  }
3795 
3796  LWLockRelease(SerializablePredicateListLock);
3797  LWLockRelease(SerializableFinishedListLock);
3798 }
3799 
3800 /*
3801  * This is the normal way to delete anything from any of the predicate
3802  * locking hash tables. Given a transaction which we know can be deleted:
3803  * delete all predicate locks held by that transaction and any predicate
3804  * lock targets which are now unreferenced by a lock; delete all conflicts
3805  * for the transaction; delete all xid values for the transaction; then
3806  * delete the transaction.
3807  *
3808  * When the partial flag is set, we can release all predicate locks and
3809  * in-conflict information -- we've established that there are no longer
3810  * any overlapping read write transactions for which this transaction could
3811  * matter -- but keep the transaction entry itself and any outConflicts.
3812  *
3813  * When the summarize flag is set, we've run short of room for sxact data
3814  * and must summarize to the SLRU. Predicate locks are transferred to a
3815  * dummy "old" transaction, with duplicate locks on a single target
3816  * collapsing to a single lock with the "latest" commitSeqNo from among
3817  * the conflicting locks..
3818  */
3819 static void
3821  bool summarize)
3822 {
3823  SERIALIZABLEXIDTAG sxidtag;
3824  dlist_mutable_iter iter;
3825 
3826  Assert(sxact != NULL);
3827  Assert(SxactIsRolledBack(sxact) || SxactIsCommitted(sxact));
3828  Assert(partial || !SxactIsOnFinishedList(sxact));
3829  Assert(LWLockHeldByMe(SerializableFinishedListLock));
3830 
3831  /*
3832  * First release all the predicate locks held by this xact (or transfer
3833  * them to OldCommittedSxact if summarize is true)
3834  */
3835  LWLockAcquire(SerializablePredicateListLock, LW_SHARED);
3836  if (IsInParallelMode())
3838  dlist_foreach_modify(iter, &sxact->predicateLocks)
3839  {
3840  PREDICATELOCK *predlock =
3841  dlist_container(PREDICATELOCK, xactLink, iter.cur);
3842  PREDICATELOCKTAG tag;
3843  PREDICATELOCKTARGET *target;
3844  PREDICATELOCKTARGETTAG targettag;
3845  uint32 targettaghash;
3846  LWLock *partitionLock;
3847 
3848  tag = predlock->tag;
3849  target = tag.myTarget;
3850  targettag = target->tag;
3851  targettaghash = PredicateLockTargetTagHashCode(&targettag);
3852  partitionLock = PredicateLockHashPartitionLock(targettaghash);
3853 
3854  LWLockAcquire(partitionLock, LW_EXCLUSIVE);
3855 
3856  dlist_delete(&predlock->targetLink);
3857 
3860  targettaghash),
3861  HASH_REMOVE, NULL);
3862  if (summarize)
3863  {
3864  bool found;
3865 
3866  /* Fold into dummy transaction list. */
3867  tag.myXact = OldCommittedSxact;
3870  targettaghash),
3871  HASH_ENTER_NULL, &found);
3872  if (!predlock)
3873  ereport(ERROR,
3874  (errcode(ERRCODE_OUT_OF_MEMORY),
3875  errmsg("out of shared memory"),
3876  errhint("You might need to increase %s.", "max_pred_locks_per_transaction")));
3877  if (found)
3878  {
3879  Assert(predlock->commitSeqNo != 0);
3881  if (predlock->commitSeqNo < sxact->commitSeqNo)
3882  predlock->commitSeqNo = sxact->commitSeqNo;
3883  }
3884  else
3885  {
3887  &predlock->targetLink);
3889  &predlock->xactLink);
3890  predlock->commitSeqNo = sxact->commitSeqNo;
3891  }
3892  }
3893  else
3894  RemoveTargetIfNoLongerUsed(target, targettaghash);
3895 
3896  LWLockRelease(partitionLock);
3897  }
3898 
3899  /*
3900  * Rather than retail removal, just re-init the head after we've run
3901  * through the list.
3902  */
3903  dlist_init(&sxact->predicateLocks);
3904 
3905  if (IsInParallelMode())
3907  LWLockRelease(SerializablePredicateListLock);
3908 
3909  sxidtag.xid = sxact->topXid;
3910  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
3911 
3912  /* Release all outConflicts (unless 'partial' is true) */
3913  if (!partial)
3914  {
3915  dlist_foreach_modify(iter, &sxact->outConflicts)
3916  {
3917  RWConflict conflict =
3918  dlist_container(RWConflictData, outLink, iter.cur);
3919 
3920  if (summarize)
3922  ReleaseRWConflict(conflict);
3923  }
3924  }
3925 
3926  /* Release all inConflicts. */
3927  dlist_foreach_modify(iter, &sxact->inConflicts)
3928  {
3929  RWConflict conflict =
3930  dlist_container(RWConflictData, inLink, iter.cur);
3931 
3932  if (summarize)
3934  ReleaseRWConflict(conflict);
3935  }
3936 
3937  /* Finally, get rid of the xid and the record of the transaction itself. */
3938  if (!partial)
3939  {
3940  if (sxidtag.xid != InvalidTransactionId)
3941  hash_search(SerializableXidHash, &sxidtag, HASH_REMOVE, NULL);
3942  ReleasePredXact(sxact);
3943  }
3944 
3945  LWLockRelease(SerializableXactHashLock);
3946 }
3947 
3948 /*
3949  * Tests whether the given top level transaction is concurrent with
3950  * (overlaps) our current transaction.
3951  *
3952  * We need to identify the top level transaction for SSI, anyway, so pass
3953  * that to this function to save the overhead of checking the snapshot's
3954  * subxip array.
3955  */
3956 static bool
3958 {
3959  Snapshot snap;
3960 
3963 
3964  snap = GetTransactionSnapshot();
3965 
3966  if (TransactionIdPrecedes(xid, snap->xmin))
3967  return false;
3968 
3969  if (TransactionIdFollowsOrEquals(xid, snap->xmax))
3970  return true;
3971 
3972  return pg_lfind32(xid, snap->xip, snap->xcnt);
3973 }
3974 
3975 bool
3977 {
3978  if (!SerializationNeededForRead(relation, snapshot))
3979  return false;
3980 
3981  /* Check if someone else has already decided that we need to die */
3983  {
3984  ereport(ERROR,
3986  errmsg("could not serialize access due to read/write dependencies among transactions"),
3987  errdetail_internal("Reason code: Canceled on identification as a pivot, during conflict out checking."),
3988  errhint("The transaction might succeed if retried.")));
3989  }
3990 
3991  return true;
3992 }
3993 
3994 /*
3995  * CheckForSerializableConflictOut
3996  * A table AM is reading a tuple that has been modified. If it determines
3997  * that the tuple version it is reading is not visible to us, it should
3998  * pass in the top level xid of the transaction that created it.
3999  * Otherwise, if it determines that it is visible to us but it has been
4000  * deleted or there is a newer version available due to an update, it
4001  * should pass in the top level xid of the modifying transaction.
4002  *
4003  * This function will check for overlap with our own transaction. If the given
4004  * xid is also serializable and the transactions overlap (i.e., they cannot see
4005  * each other's writes), then we have a conflict out.
4006  */
4007 void
4009 {
4010  SERIALIZABLEXIDTAG sxidtag;
4011  SERIALIZABLEXID *sxid;
4012  SERIALIZABLEXACT *sxact;
4013 
4014  if (!SerializationNeededForRead(relation, snapshot))
4015  return;
4016 
4017  /* Check if someone else has already decided that we need to die */
4019  {
4020  ereport(ERROR,
4022  errmsg("could not serialize access due to read/write dependencies among transactions"),
4023  errdetail_internal("Reason code: Canceled on identification as a pivot, during conflict out checking."),
4024  errhint("The transaction might succeed if retried.")));
4025  }
4027 
4029  return;
4030 
4031  /*
4032  * Find sxact or summarized info for the top level xid.
4033  */
4034  sxidtag.xid = xid;
4035  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
4036  sxid = (SERIALIZABLEXID *)
4037  hash_search(SerializableXidHash, &sxidtag, HASH_FIND, NULL);
4038  if (!sxid)
4039  {
4040  /*
4041  * Transaction not found in "normal" SSI structures. Check whether it
4042  * got pushed out to SLRU storage for "old committed" transactions.
4043  */
4044  SerCommitSeqNo conflictCommitSeqNo;
4045 
4046  conflictCommitSeqNo = SerialGetMinConflictCommitSeqNo(xid);
4047  if (conflictCommitSeqNo != 0)
4048  {
4049  if (conflictCommitSeqNo != InvalidSerCommitSeqNo
4051  || conflictCommitSeqNo
4053  ereport(ERROR,
4055  errmsg("could not serialize access due to read/write dependencies among transactions"),
4056  errdetail_internal("Reason code: Canceled on conflict out to old pivot %u.", xid),
4057  errhint("The transaction might succeed if retried.")));
4058 
4061  ereport(ERROR,
4063  errmsg("could not serialize access due to read/write dependencies among transactions"),
4064  errdetail_internal("Reason code: Canceled on identification as a pivot, with conflict out to old committed transaction %u.", xid),
4065  errhint("The transaction might succeed if retried.")));
4066 
4068  }
4069 
4070  /* It's not serializable or otherwise not important. */
4071  LWLockRelease(SerializableXactHashLock);
4072  return;
4073  }
4074  sxact = sxid->myXact;
4075  Assert(TransactionIdEquals(sxact->topXid, xid));
4076  if (sxact == MySerializableXact || SxactIsDoomed(sxact))
4077  {
4078  /* Can't conflict with ourself or a transaction that will roll back. */
4079  LWLockRelease(SerializableXactHashLock);
4080  return;
4081  }
4082 
4083  /*
4084  * We have a conflict out to a transaction which has a conflict out to a
4085  * summarized transaction. That summarized transaction must have
4086  * committed first, and we can't tell when it committed in relation to our
4087  * snapshot acquisition, so something needs to be canceled.
4088  */
4089  if (SxactHasSummaryConflictOut(sxact))
4090  {
4091  if (!SxactIsPrepared(sxact))
4092  {
4093  sxact->flags |= SXACT_FLAG_DOOMED;
4094  LWLockRelease(SerializableXactHashLock);
4095  return;
4096  }
4097  else
4098  {
4099  LWLockRelease(SerializableXactHashLock);
4100  ereport(ERROR,
4102  errmsg("could not serialize access due to read/write dependencies among transactions"),
4103  errdetail_internal("Reason code: Canceled on conflict out to old pivot."),
4104  errhint("The transaction might succeed if retried.")));
4105  }
4106  }
4107 
4108  /*
4109  * If this is a read-only transaction and the writing transaction has
4110  * committed, and it doesn't have a rw-conflict to a transaction which
4111  * committed before it, no conflict.
4112  */
4114  && SxactIsCommitted(sxact)
4115  && !SxactHasSummaryConflictOut(sxact)
4116  && (!SxactHasConflictOut(sxact)
4118  {
4119  /* Read-only transaction will appear to run first. No conflict. */
4120  LWLockRelease(SerializableXactHashLock);
4121  return;
4122  }
4123 
4124  if (!XidIsConcurrent(xid))
4125  {
4126  /* This write was already in our snapshot; no conflict. */
4127  LWLockRelease(SerializableXactHashLock);
4128  return;
4129  }
4130 
4132  {
4133  /* We don't want duplicate conflict records in the list. */
4134  LWLockRelease(SerializableXactHashLock);
4135  return;
4136  }
4137 
4138  /*
4139  * Flag the conflict. But first, if this conflict creates a dangerous
4140  * structure, ereport an error.
4141  */
4143  LWLockRelease(SerializableXactHashLock);
4144 }
4145 
4146 /*
4147  * Check a particular target for rw-dependency conflict in. A subroutine of
4148  * CheckForSerializableConflictIn().
4149  */
4150 static void
4152 {
4153  uint32 targettaghash;
4154  LWLock *partitionLock;
4155  PREDICATELOCKTARGET *target;
4156  PREDICATELOCK *mypredlock = NULL;
4157  PREDICATELOCKTAG mypredlocktag;
4158  dlist_mutable_iter iter;
4159 
4161 
4162  /*
4163  * The same hash and LW lock apply to the lock target and the lock itself.
4164  */
4165  targettaghash = PredicateLockTargetTagHashCode(targettag);
4166  partitionLock = PredicateLockHashPartitionLock(targettaghash);
4167  LWLockAcquire(partitionLock, LW_SHARED);
4168  target = (PREDICATELOCKTARGET *)
4170  targettag, targettaghash,
4171  HASH_FIND, NULL);
4172  if (!target)
4173  {
4174  /* Nothing has this target locked; we're done here. */
4175  LWLockRelease(partitionLock);
4176  return;
4177  }
4178 
4179  /*
4180  * Each lock for an overlapping transaction represents a conflict: a
4181  * rw-dependency in to this transaction.
4182  */
4183  LWLockAcquire(SerializableXactHashLock, LW_SHARED);
4184 
4185  dlist_foreach_modify(iter, &target->predicateLocks)
4186  {
4187  PREDICATELOCK *predlock =
4188  dlist_container(PREDICATELOCK, targetLink, iter.cur);
4189  SERIALIZABLEXACT *sxact = predlock->tag.myXact;
4190 
4191  if (sxact == MySerializableXact)
4192  {
4193  /*
4194  * If we're getting a write lock on a tuple, we don't need a
4195  * predicate (SIREAD) lock on the same tuple. We can safely remove
4196  * our SIREAD lock, but we'll defer doing so until after the loop
4197  * because that requires upgrading to an exclusive partition lock.
4198  *
4199  * We can't use this optimization within a subtransaction because
4200  * the subtransaction could roll back, and we would be left
4201  * without any lock at the top level.
4202  */
4203  if (!IsSubTransaction()
4204  && GET_PREDICATELOCKTARGETTAG_OFFSET(*targettag))
4205  {
4206  mypredlock = predlock;
4207  mypredlocktag = predlock->tag;
4208  }
4209  }
4210  else if (!SxactIsDoomed(sxact)
4211  && (!SxactIsCommitted(sxact)
4213  sxact->finishedBefore))
4215  {
4216  LWLockRelease(SerializableXactHashLock);
4217  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
4218 
4219  /*
4220  * Re-check after getting exclusive lock because the other
4221  * transaction may have flagged a conflict.
4222  */
4223  if (!SxactIsDoomed(sxact)
4224  && (!SxactIsCommitted(sxact)
4226  sxact->finishedBefore))
4228  {
4230  }
4231 
4232  LWLockRelease(SerializableXactHashLock);
4233  LWLockAcquire(SerializableXactHashLock, LW_SHARED);
4234  }
4235  }
4236  LWLockRelease(SerializableXactHashLock);
4237  LWLockRelease(partitionLock);
4238 
4239  /*
4240  * If we found one of our own SIREAD locks to remove, remove it now.
4241  *
4242  * At this point our transaction already has a RowExclusiveLock on the
4243  * relation, so we are OK to drop the predicate lock on the tuple, if
4244  * found, without fearing that another write against the tuple will occur
4245  * before the MVCC information makes it to the buffer.
4246  */
4247  if (mypredlock != NULL)
4248  {
4249  uint32 predlockhashcode;
4250  PREDICATELOCK *rmpredlock;
4251 
4252  LWLockAcquire(SerializablePredicateListLock, LW_SHARED);
4253  if (IsInParallelMode())
4255  LWLockAcquire(partitionLock, LW_EXCLUSIVE);
4256  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
4257 
4258  /*
4259  * Remove the predicate lock from shared memory, if it wasn't removed
4260  * while the locks were released. One way that could happen is from
4261  * autovacuum cleaning up an index.
4262  */
4263  predlockhashcode = PredicateLockHashCodeFromTargetHashCode
4264  (&mypredlocktag, targettaghash);
4265  rmpredlock = (PREDICATELOCK *)
4267  &mypredlocktag,
4268  predlockhashcode,
4269  HASH_FIND, NULL);
4270  if (rmpredlock != NULL)
4271  {
4272  Assert(rmpredlock == mypredlock);
4273 
4274  dlist_delete(&(mypredlock->targetLink));
4275  dlist_delete(&(mypredlock->xactLink));
4276 
4277  rmpredlock = (PREDICATELOCK *)
4279  &mypredlocktag,
4280  predlockhashcode,
4281  HASH_REMOVE, NULL);
4282  Assert(rmpredlock == mypredlock);
4283 
4284  RemoveTargetIfNoLongerUsed(target, targettaghash);
4285  }
4286 
4287  LWLockRelease(SerializableXactHashLock);
4288  LWLockRelease(partitionLock);
4289  if (IsInParallelMode())
4291  LWLockRelease(SerializablePredicateListLock);
4292 
4293  if (rmpredlock != NULL)
4294  {
4295  /*
4296  * Remove entry in local lock table if it exists. It's OK if it
4297  * doesn't exist; that means the lock was transferred to a new
4298  * target by a different backend.
4299  */
4301  targettag, targettaghash,
4302  HASH_REMOVE, NULL);
4303 
4304  DecrementParentLocks(targettag);
4305  }
4306  }
4307 }
4308 
4309 /*
4310  * CheckForSerializableConflictIn
4311  * We are writing the given tuple. If that indicates a rw-conflict
4312  * in from another serializable transaction, take appropriate action.
4313  *
4314  * Skip checking for any granularity for which a parameter is missing.
4315  *
4316  * A tuple update or delete is in conflict if we have a predicate lock
4317  * against the relation or page in which the tuple exists, or against the
4318  * tuple itself.
4319  */
4320 void
4322 {
4323  PREDICATELOCKTARGETTAG targettag;
4324 
4325  if (!SerializationNeededForWrite(relation))
4326  return;
4327 
4328  /* Check if someone else has already decided that we need to die */
4330  ereport(ERROR,
4332  errmsg("could not serialize access due to read/write dependencies among transactions"),
4333  errdetail_internal("Reason code: Canceled on identification as a pivot, during conflict in checking."),
4334  errhint("The transaction might succeed if retried.")));
4335 
4336  /*
4337  * We're doing a write which might cause rw-conflicts now or later.
4338  * Memorize that fact.
4339  */
4340  MyXactDidWrite = true;
4341 
4342  /*
4343  * It is important that we check for locks from the finest granularity to
4344  * the coarsest granularity, so that granularity promotion doesn't cause
4345  * us to miss a lock. The new (coarser) lock will be acquired before the
4346  * old (finer) locks are released.
4347  *
4348  * It is not possible to take and hold a lock across the checks for all
4349  * granularities because each target could be in a separate partition.
4350  */
4351  if (tid != NULL)
4352  {
4354  relation->rd_locator.dbOid,
4355  relation->rd_id,
4358  CheckTargetForConflictsIn(&targettag);
4359  }
4360 
4361  if (blkno != InvalidBlockNumber)
4362  {
4364  relation->rd_locator.dbOid,
4365  relation->rd_id,
4366  blkno);
4367  CheckTargetForConflictsIn(&targettag);
4368  }
4369 
4371  relation->rd_locator.dbOid,
4372  relation->rd_id);
4373  CheckTargetForConflictsIn(&targettag);
4374 }
4375 
4376 /*
4377  * CheckTableForSerializableConflictIn
4378  * The entire table is going through a DDL-style logical mass delete
4379  * like TRUNCATE or DROP TABLE. If that causes a rw-conflict in from
4380  * another serializable transaction, take appropriate action.
4381  *
4382  * While these operations do not operate entirely within the bounds of
4383  * snapshot isolation, they can occur inside a serializable transaction, and
4384  * will logically occur after any reads which saw rows which were destroyed
4385  * by these operations, so we do what we can to serialize properly under
4386  * SSI.
4387  *
4388  * The relation passed in must be a heap relation. Any predicate lock of any
4389  * granularity on the heap will cause a rw-conflict in to this transaction.
4390  * Predicate locks on indexes do not matter because they only exist to guard
4391  * against conflicting inserts into the index, and this is a mass *delete*.
4392  * When a table is truncated or dropped, the index will also be truncated
4393  * or dropped, and we'll deal with locks on the index when that happens.
4394  *
4395  * Dropping or truncating a table also needs to drop any existing predicate
4396  * locks on heap tuples or pages, because they're about to go away. This
4397  * should be done before altering the predicate locks because the transaction
4398  * could be rolled back because of a conflict, in which case the lock changes
4399  * are not needed. (At the moment, we don't actually bother to drop the
4400  * existing locks on a dropped or truncated table at the moment. That might
4401  * lead to some false positives, but it doesn't seem worth the trouble.)
4402  */
4403 void
4405 {
4406  HASH_SEQ_STATUS seqstat;
4407  PREDICATELOCKTARGET *target;
4408  Oid dbId;
4409  Oid heapId;
4410  int i;
4411 
4412  /*
4413  * Bail out quickly if there are no serializable transactions running.
4414  * It's safe to check this without taking locks because the caller is
4415  * holding an ACCESS EXCLUSIVE lock on the relation. No new locks which
4416  * would matter here can be acquired while that is held.
4417  */
4419  return;
4420 
4421  if (!SerializationNeededForWrite(relation))
4422  return;
4423 
4424  /*
4425  * We're doing a write which might cause rw-conflicts now or later.
4426  * Memorize that fact.
4427  */
4428  MyXactDidWrite = true;
4429 
4430  Assert(relation->rd_index == NULL); /* not an index relation */
4431 
4432  dbId = relation->rd_locator.dbOid;
4433  heapId = relation->rd_id;
4434 
4435  LWLockAcquire(SerializablePredicateListLock, LW_EXCLUSIVE);
4436  for (i = 0; i < NUM_PREDICATELOCK_PARTITIONS; i++)
4438  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
4439 
4440  /* Scan through target list */
4442 
4443  while ((target = (PREDICATELOCKTARGET *) hash_seq_search(&seqstat)))
4444  {
4445  dlist_mutable_iter iter;
4446 
4447  /*
4448  * Check whether this is a target which needs attention.
4449  */
4450  if (GET_PREDICATELOCKTARGETTAG_RELATION(target->tag) != heapId)
4451  continue; /* wrong relation id */
4452  if (GET_PREDICATELOCKTARGETTAG_DB(target->tag) != dbId)
4453  continue; /* wrong database id */
4454 
4455  /*
4456  * Loop through locks for this target and flag conflicts.
4457  */
4458  dlist_foreach_modify(iter, &target->predicateLocks)
4459  {
4460  PREDICATELOCK *predlock =
4461  dlist_container(PREDICATELOCK, targetLink, iter.cur);
4462 
4463  if (predlock->tag.myXact != MySerializableXact
4465  {
4467  }
4468  }
4469  }
4470 
4471  /* Release locks in reverse order */
4472  LWLockRelease(SerializableXactHashLock);
4473  for (i = NUM_PREDICATELOCK_PARTITIONS - 1; i >= 0; i--)
4475  LWLockRelease(SerializablePredicateListLock);
4476 }
4477 
4478 
4479 /*
4480  * Flag a rw-dependency between two serializable transactions.
4481  *
4482  * The caller is responsible for ensuring that we have a LW lock on
4483  * the transaction hash table.
4484  */
4485 static void
4487 {
4488  Assert(reader != writer);
4489 
4490  /* First, see if this conflict causes failure. */
4492 
4493  /* Actually do the conflict flagging. */
4494  if (reader == OldCommittedSxact)
4496  else if (writer == OldCommittedSxact)
4498  else
4499  SetRWConflict(reader, writer);
4500 }
4501 
4502 /*----------------------------------------------------------------------------
4503  * We are about to add a RW-edge to the dependency graph - check that we don't
4504  * introduce a dangerous structure by doing so, and abort one of the
4505  * transactions if so.
4506  *
4507  * A serialization failure can only occur if there is a dangerous structure
4508  * in the dependency graph:
4509  *
4510  * Tin ------> Tpivot ------> Tout
4511  * rw rw
4512  *
4513  * Furthermore, Tout must commit first.
4514  *
4515  * One more optimization is that if Tin is declared READ ONLY (or commits
4516  * without writing), we can only have a problem if Tout committed before Tin
4517  * acquired its snapshot.
4518  *----------------------------------------------------------------------------
4519  */
4520 static void
4522  SERIALIZABLEXACT *writer)
4523 {
4524  bool failure;
4525 
4526  Assert(LWLockHeldByMe(SerializableXactHashLock));
4527 
4528  failure = false;
4529 
4530  /*------------------------------------------------------------------------
4531  * Check for already-committed writer with rw-conflict out flagged
4532  * (conflict-flag on W means that T2 committed before W):
4533  *
4534  * R ------> W ------> T2
4535  * rw rw
4536  *
4537  * That is a dangerous structure, so we must abort. (Since the writer
4538  * has already committed, we must be the reader)
4539  *------------------------------------------------------------------------
4540  */
4541  if (SxactIsCommitted(writer)
4542  && (SxactHasConflictOut(writer) || SxactHasSummaryConflictOut(writer)))
4543  failure = true;
4544 
4545  /*------------------------------------------------------------------------
4546  * Check whether the writer has become a pivot with an out-conflict
4547  * committed transaction (T2), and T2 committed first:
4548  *
4549  * R ------> W ------> T2
4550  * rw rw
4551  *
4552  * Because T2 must've committed first, there is no anomaly if:
4553  * - the reader committed before T2
4554  * - the writer committed before T2
4555  * - the reader is a READ ONLY transaction and the reader was concurrent
4556  * with T2 (= reader acquired its snapshot before T2 committed)
4557  *
4558  * We also handle the case that T2 is prepared but not yet committed
4559  * here. In that case T2 has already checked for conflicts, so if it
4560  * commits first, making the above conflict real, it's too late for it
4561  * to abort.
4562  *------------------------------------------------------------------------
4563  */
4564  if (!failure && SxactHasSummaryConflictOut(writer))
4565  failure = true;
4566  else if (!failure)
4567  {
4568  dlist_iter iter;
4569 
4570  dlist_foreach(iter, &writer->outConflicts)
4571  {
4572  RWConflict conflict =
4573  dlist_container(RWConflictData, outLink, iter.cur);
4574  SERIALIZABLEXACT *t2 = conflict->sxactIn;
4575 
4576  if (SxactIsPrepared(t2)
4577  && (!SxactIsCommitted(reader)
4578  || t2->prepareSeqNo <= reader->commitSeqNo)
4579  && (!SxactIsCommitted(writer)
4580  || t2->prepareSeqNo <= writer->commitSeqNo)
4581  && (!SxactIsReadOnly(reader)
4582  || t2->prepareSeqNo <= reader->SeqNo.lastCommitBeforeSnapshot))
4583  {
4584  failure = true;
4585  break;
4586  }
4587  }
4588  }
4589 
4590  /*------------------------------------------------------------------------
4591  * Check whether the reader has become a pivot with a writer
4592  * that's committed (or prepared):
4593  *
4594  * T0 ------> R ------> W
4595  * rw rw
4596  *
4597  * Because W must've committed first for an anomaly to occur, there is no
4598  * anomaly if:
4599  * - T0 committed before the writer
4600  * - T0 is READ ONLY, and overlaps the writer
4601  *------------------------------------------------------------------------
4602  */
4603  if (!failure && SxactIsPrepared(writer) && !SxactIsReadOnly(reader))
4604  {
4605  if (SxactHasSummaryConflictIn(reader))
4606  {
4607  failure = true;
4608  }
4609  else
4610  {
4611  dlist_iter iter;
4612 
4613  /*
4614  * The unconstify is needed as we have no const version of
4615  * dlist_foreach().
4616  */
4617  dlist_foreach(iter, &unconstify(SERIALIZABLEXACT *, reader)->inConflicts)
4618  {
4619  const RWConflict conflict =
4620  dlist_container(RWConflictData, inLink, iter.cur);
4621  const SERIALIZABLEXACT *t0 = conflict->sxactOut;
4622 
4623  if (!SxactIsDoomed(t0)
4624  && (!SxactIsCommitted(t0)
4625  || t0->commitSeqNo >= writer->prepareSeqNo)
4626  && (!SxactIsReadOnly(t0)
4627  || t0->SeqNo.lastCommitBeforeSnapshot >= writer->prepareSeqNo))
4628  {
4629  failure = true;
4630  break;
4631  }
4632  }
4633  }
4634  }
4635 
4636  if (failure)
4637  {
4638  /*
4639  * We have to kill a transaction to avoid a possible anomaly from
4640  * occurring. If the writer is us, we can just ereport() to cause a
4641  * transaction abort. Otherwise we flag the writer for termination,
4642  * causing it to abort when it tries to commit. However, if the writer
4643  * is a prepared transaction, already prepared, we can't abort it
4644  * anymore, so we have to kill the reader instead.
4645  */
4646  if (MySerializableXact == writer)
4647  {
4648  LWLockRelease(SerializableXactHashLock);
4649  ereport(ERROR,
4651  errmsg("could not serialize access due to read/write dependencies among transactions"),
4652  errdetail_internal("Reason code: Canceled on identification as a pivot, during write."),
4653  errhint("The transaction might succeed if retried.")));
4654  }
4655  else if (SxactIsPrepared(writer))
4656  {
4657  LWLockRelease(SerializableXactHashLock);
4658 
4659  /* if we're not the writer, we have to be the reader */
4660  Assert(MySerializableXact == reader);
4661  ereport(ERROR,
4663  errmsg("could not serialize access due to read/write dependencies among transactions"),
4664  errdetail_internal("Reason code: Canceled on conflict out to pivot %u, during read.", writer->topXid),
4665  errhint("The transaction might succeed if retried.")));
4666  }
4667  writer->flags |= SXACT_FLAG_DOOMED;
4668  }
4669 }
4670 
4671 /*
4672  * PreCommit_CheckForSerializationFailure
4673  * Check for dangerous structures in a serializable transaction
4674  * at commit.
4675  *
4676  * We're checking for a dangerous structure as each conflict is recorded.
4677  * The only way we could have a problem at commit is if this is the "out"
4678  * side of a pivot, and neither the "in" side nor the pivot has yet
4679  * committed.
4680  *
4681  * If a dangerous structure is found, the pivot (the near conflict) is
4682  * marked for death, because rolling back another transaction might mean
4683  * that we fail without ever making progress. This transaction is
4684  * committing writes, so letting it commit ensures progress. If we
4685  * canceled the far conflict, it might immediately fail again on retry.
4686  */
4687 void
4689 {
4690  dlist_iter near_iter;
4691 
4693  return;
4694 
4696 
4697  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
4698 
4699  /*
4700  * Check if someone else has already decided that we need to die. Since
4701  * we set our own DOOMED flag when partially releasing, ignore in that
4702  * case.
4703  */
4706  {
4707  LWLockRelease(SerializableXactHashLock);
4708  ereport(ERROR,
4710  errmsg("could not serialize access due to read/write dependencies among transactions"),
4711  errdetail_internal("Reason code: Canceled on identification as a pivot, during commit attempt."),
4712  errhint("The transaction might succeed if retried.")));
4713  }
4714 
4716  {
4717  RWConflict nearConflict =
4718  dlist_container(RWConflictData, inLink, near_iter.cur);
4719 
4720  if (!SxactIsCommitted(nearConflict->sxactOut)
4721  && !SxactIsDoomed(nearConflict->sxactOut))
4722  {
4723  dlist_iter far_iter;
4724 
4725  dlist_foreach(far_iter, &nearConflict->sxactOut->inConflicts)
4726  {
4727  RWConflict farConflict =
4728  dlist_container(RWConflictData, inLink, far_iter.cur);
4729 
4730  if (farConflict->sxactOut == MySerializableXact
4731  || (!SxactIsCommitted(farConflict->sxactOut)
4732  && !SxactIsReadOnly(farConflict->sxactOut)
4733  && !SxactIsDoomed(farConflict->sxactOut)))
4734  {
4735  /*
4736  * Normally, we kill the pivot transaction to make sure we
4737  * make progress if the failing transaction is retried.
4738  * However, we can't kill it if it's already prepared, so
4739  * in that case we commit suicide instead.
4740  */
4741  if (SxactIsPrepared(nearConflict->sxactOut))
4742  {
4743  LWLockRelease(SerializableXactHashLock);
4744  ereport(ERROR,
4746  errmsg("could not serialize access due to read/write dependencies among transactions"),
4747  errdetail_internal("Reason code: Canceled on commit attempt with conflict in from prepared pivot."),
4748  errhint("The transaction might succeed if retried.")));
4749  }
4750  nearConflict->sxactOut->flags |= SXACT_FLAG_DOOMED;
4751  break;
4752  }
4753  }
4754  }
4755  }
4756 
4759 
4760  LWLockRelease(SerializableXactHashLock);
4761 }
4762 
4763 /*------------------------------------------------------------------------*/
4764 
4765 /*
4766  * Two-phase commit support
4767  */
4768 
4769 /*
4770  * AtPrepare_Locks
4771  * Do the preparatory work for a PREPARE: make 2PC state file
4772  * records for all predicate locks currently held.
4773  */
4774 void
4776 {
4777  SERIALIZABLEXACT *sxact;
4778  TwoPhasePredicateRecord record;
4779  TwoPhasePredicateXactRecord *xactRecord;
4780  TwoPhasePredicateLockRecord *lockRecord;
4781  dlist_iter iter;
4782 
4783  sxact = MySerializableXact;
4784  xactRecord = &(record.data.xactRecord);
4785  lockRecord = &(record.data.lockRecord);
4786 
4788  return;
4789 
4790  /* Generate an xact record for our SERIALIZABLEXACT */
4792  xactRecord->xmin = MySerializableXact->xmin;
4793  xactRecord->flags = MySerializableXact->flags;
4794 
4795  /*
4796  * Note that we don't include the list of conflicts in our out in the
4797  * statefile, because new conflicts can be added even after the
4798  * transaction prepares. We'll just make a conservative assumption during
4799  * recovery instead.
4800  */
4801 
4803  &record, sizeof(record));
4804 
4805  /*
4806  * Generate a lock record for each lock.
4807  *
4808  * To do this, we need to walk the predicate lock list in our sxact rather
4809  * than using the local predicate lock table because the latter is not
4810  * guaranteed to be accurate.
4811  */
4812  LWLockAcquire(SerializablePredicateListLock, LW_SHARED);
4813 
4814  /*
4815  * No need to take sxact->perXactPredicateListLock in parallel mode
4816  * because there cannot be any parallel workers running while we are
4817  * preparing a transaction.
4818  */
4820 
4821  dlist_foreach(iter, &sxact->predicateLocks)
4822  {
4823  PREDICATELOCK *predlock =
4824  dlist_container(PREDICATELOCK, xactLink, iter.cur);
4825 
4827  lockRecord->target = predlock->tag.myTarget->tag;
4828 
4830  &record, sizeof(record));
4831  }
4832 
4833  LWLockRelease(SerializablePredicateListLock);
4834 }
4835 
4836 /*
4837  * PostPrepare_Locks
4838  * Clean up after successful PREPARE. Unlike the non-predicate
4839  * lock manager, we do not need to transfer locks to a dummy
4840  * PGPROC because our SERIALIZABLEXACT will stay around
4841  * anyway. We only need to clean up our local state.
4842  */
4843 void
4845 {
4847  return;
4848 
4850 
4851  MySerializableXact->pid = 0;
4853 
4855  LocalPredicateLockHash = NULL;
4856 
4858  MyXactDidWrite = false;
4859 }
4860 
4861 /*
4862  * PredicateLockTwoPhaseFinish
4863  * Release a prepared transaction's predicate locks once it
4864  * commits or aborts.
4865  */
4866 void
4868 {
4869  SERIALIZABLEXID *sxid;
4870  SERIALIZABLEXIDTAG sxidtag;
4871 
4872  sxidtag.xid = xid;
4873 
4874  LWLockAcquire(SerializableXactHashLock, LW_SHARED);
4875  sxid = (SERIALIZABLEXID *)
4876  hash_search(SerializableXidHash, &sxidtag, HASH_FIND, NULL);
4877  LWLockRelease(SerializableXactHashLock);
4878 
4879  /* xid will not be found if it wasn't a serializable transaction */
4880  if (sxid == NULL)
4881  return;
4882 
4883  /* Release its locks */
4884  MySerializableXact = sxid->myXact;
4885  MyXactDidWrite = true; /* conservatively assume that we wrote
4886  * something */
4887  ReleasePredicateLocks(isCommit, false);
4888 }
4889 
4890 /*
4891  * Re-acquire a predicate lock belonging to a transaction that was prepared.
4892  */
4893 void
4895  void *recdata, uint32 len)
4896 {
4897  TwoPhasePredicateRecord *record;
4898 
4899  Assert(len == sizeof(TwoPhasePredicateRecord));
4900 
4901  record = (TwoPhasePredicateRecord *) recdata;
4902 
4903  Assert((record->type == TWOPHASEPREDICATERECORD_XACT) ||
4904  (record->type == TWOPHASEPREDICATERECORD_LOCK));
4905 
4906  if (record->type == TWOPHASEPREDICATERECORD_XACT)
4907  {
4908  /* Per-transaction record. Set up a SERIALIZABLEXACT. */
4909  TwoPhasePredicateXactRecord *xactRecord;
4910  SERIALIZABLEXACT *sxact;
4911  SERIALIZABLEXID *sxid;
4912  SERIALIZABLEXIDTAG sxidtag;
4913  bool found;
4914 
4915  xactRecord = (TwoPhasePredicateXactRecord *) &record->data.xactRecord;
4916 
4917  LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
4918  sxact = CreatePredXact();
4919  if (!sxact)
4920  ereport(ERROR,
4921  (errcode(ERRCODE_OUT_OF_MEMORY),
4922  errmsg("out of shared memory")));
4923 
4924  /* vxid for a prepared xact is INVALID_PROC_NUMBER/xid; no pid */
4927  sxact->pid = 0;
4928  sxact->pgprocno = INVALID_PROC_NUMBER;
4929 
4930  /* a prepared xact hasn't committed yet */
4934 
4936 
4937  /*
4938  * Don't need to track this; no transactions running at the time the
4939  * recovered xact started are still active, except possibly other
4940  * prepared xacts and we don't care whether those are RO_SAFE or not.
4941  */
4943 
4944  dlist_init(&(sxact->predicateLocks));
4945  dlist_node_init(&sxact->finishedLink);
4946 
4947  sxact->topXid = xid;
4948  sxact->xmin = xactRecord->xmin;
4949  sxact->flags = xactRecord->flags;
4950  Assert(SxactIsPrepared(sxact));
4951  if (!SxactIsReadOnly(sxact))
4952  {
4956  }
4957 
4958  /*
4959  * We don't know whether the transaction had any conflicts or not, so
4960  * we'll conservatively assume that it had both a conflict in and a
4961  * conflict out, and represent that with the summary conflict flags.
4962  */
4963  dlist_init(&(sxact->outConflicts));
4964  dlist_init(&(sxact->inConflicts));
4967 
4968  /* Register the transaction's xid */
4969  sxidtag.xid = xid;
4971  &sxidtag,
4972  HASH_ENTER, &found);
4973  Assert(sxid != NULL);
4974  Assert(!found);
4975  sxid->myXact = (SERIALIZABLEXACT *) sxact;
4976 
4977  /*
4978  * Update global xmin. Note that this is a special case compared to
4979  * registering a normal transaction, because the global xmin might go
4980  * backwards. That's OK, because until recovery is over we're not
4981  * going to complete any transactions or create any non-prepared
4982  * transactions, so there's no danger of throwing away.
4983  */
4986  {
4987  PredXact->SxactGlobalXmin = sxact->xmin;
4989  SerialSetActiveSerXmin(sxact->xmin);
4990  }
4991  else if (TransactionIdEquals(sxact->xmin, PredXact->SxactGlobalXmin))
4992  {
4995  }
4996 
4997  LWLockRelease(SerializableXactHashLock);
4998  }
4999  else if (record->type == TWOPHASEPREDICATERECORD_LOCK)
5000  {
5001  /* Lock record. Recreate the PREDICATELOCK */
5002  TwoPhasePredicateLockRecord *lockRecord;
5003  SERIALIZABLEXID *sxid;
5004  SERIALIZABLEXACT *sxact;
5005  SERIALIZABLEXIDTAG sxidtag;
5006  uint32 targettaghash;
5007 
5008  lockRecord = (TwoPhasePredicateLockRecord *) &record->data.lockRecord;
5009  targettaghash = PredicateLockTargetTagHashCode(&lockRecord->target);
5010 
5011  LWLockAcquire(SerializableXactHashLock, LW_SHARED);
5012  sxidtag.xid = xid;
5013  sxid = (SERIALIZABLEXID *)
5014  hash_search(SerializableXidHash, &sxidtag, HASH_FIND, NULL);
5015  LWLockRelease(SerializableXactHashLock);
5016 
5017  Assert(sxid != NULL);
5018  sxact = sxid->myXact;
5019  Assert(sxact != InvalidSerializableXact);
5020 
5021  CreatePredicateLock(&lockRecord->target, targettaghash, sxact);
5022  }
5023 }
5024 
5025 /*
5026  * Prepare to share the current SERIALIZABLEXACT with parallel workers.
5027  * Return a handle object that can be used by AttachSerializableXact() in a
5028  * parallel worker.
5029  */
5032 {
5033  return MySerializableXact;
5034 }
5035 
5036 /*
5037  * Allow parallel workers to import the leader's SERIALIZABLEXACT.
5038  */
5039 void
5041 {
5042 
5044 
5045  MySerializableXact = (SERIALIZABLEXACT *) handle;
5048 }
bool ParallelContextActive(void)
Definition: parallel.c:1003
uint32 BlockNumber
Definition: block.h:31
#define InvalidBlockNumber
Definition: block.h:33
static bool BlockNumberIsValid(BlockNumber blockNumber)
Definition: block.h:71
unsigned short uint16
Definition: c.h:505
#define unconstify(underlying_type, expr)
Definition: c.h:1245
unsigned int uint32
Definition: c.h:506
#define PG_USED_FOR_ASSERTS_ONLY
Definition: c.h:182
#define Assert(condition)
Definition: c.h:858
uint32 LocalTransactionId
Definition: c.h:654
uint32 TransactionId
Definition: c.h:652
size_t Size
Definition: c.h:605
void hash_destroy(HTAB *hashp)
Definition: dynahash.c:865
void * hash_search(HTAB *hashp, const void *keyPtr, HASHACTION action, bool *foundPtr)
Definition: dynahash.c:955
HTAB * hash_create(const char *tabname, long nelem, const HASHCTL *info, int flags)
Definition: dynahash.c:352
long hash_get_num_entries(HTAB *hashp)
Definition: dynahash.c:1341
Size hash_estimate_size(long num_entries, Size entrysize)
Definition: dynahash.c:783
void * hash_search_with_hash_value(HTAB *hashp, const void *keyPtr, uint32 hashvalue, HASHACTION action, bool *foundPtr)
Definition: dynahash.c:968
void * hash_seq_search(HASH_SEQ_STATUS *status)
Definition: dynahash.c:1395
void hash_seq_init(HASH_SEQ_STATUS *status, HTAB *hashp)
Definition: dynahash.c:1385
int errmsg_internal(const char *fmt,...)
Definition: elog.c:1159
int errdetail_internal(const char *fmt,...)
Definition: elog.c:1232
int errdetail(const char *fmt,...)
Definition: elog.c:1205
int errhint(const char *fmt,...)
Definition: elog.c:1319
int errcode(int sqlerrcode)
Definition: elog.c:859
int errmsg(const char *fmt,...)
Definition: elog.c:1072
#define DEBUG2
Definition: elog.h:29
#define ERROR
Definition: elog.h:39
#define elog(elevel,...)
Definition: elog.h:224
#define ereport(elevel,...)
Definition: elog.h:149
int MyProcPid
Definition: globals.c:45
ProcNumber MyProcNumber
Definition: globals.c:87
bool IsUnderPostmaster
Definition: globals.c:117
int MaxBackends
Definition: globals.c:143
int serializable_buffers
Definition: globals.c:166
#define newval
GucSource
Definition: guc.h:108
@ HASH_FIND
Definition: hsearch.h:113
@ HASH_REMOVE
Definition: hsearch.h:115
@ HASH_ENTER
Definition: hsearch.h:114
@ HASH_ENTER_NULL
Definition: hsearch.h:116
#define HASH_ELEM
Definition: hsearch.h:95
#define HASH_FUNCTION
Definition: hsearch.h:98
#define HASH_BLOBS
Definition: hsearch.h:97
#define HASH_FIXED_SIZE
Definition: hsearch.h:105
#define HASH_PARTITION
Definition: hsearch.h:92
#define dlist_foreach(iter, lhead)
Definition: ilist.h:623
static void dlist_init(dlist_head *head)
Definition: ilist.h:314
#define dlist_head_element(type, membername, lhead)
Definition: ilist.h:603
static void dlist_delete_thoroughly(dlist_node *node)
Definition: ilist.h:416
static void dlist_delete(dlist_node *node)
Definition: ilist.h:405
static dlist_node * dlist_pop_head_node(dlist_head *head)
Definition: ilist.h:450
#define dlist_foreach_modify(iter, lhead)
Definition: ilist.h:640
static bool dlist_is_empty(const dlist_head *head)
Definition: ilist.h:336
static void dlist_push_tail(dlist_head *head, dlist_node *node)
Definition: ilist.h:364
static void dlist_node_init(dlist_node *node)
Definition: ilist.h:325
#define dlist_container(type, membername, ptr)
Definition: ilist.h:593
#define IsParallelWorker()
Definition: parallel.h:60
FILE * output
long val
Definition: informix.c:670
static bool success
Definition: initdb.c:186
int i
Definition: isn.c:73
static OffsetNumber ItemPointerGetOffsetNumber(const ItemPointerData *pointer)
Definition: itemptr.h:124
static BlockNumber ItemPointerGetBlockNumber(const ItemPointerData *pointer)
Definition: itemptr.h:103
exit(1)
#define GET_VXID_FROM_PGPROC(vxid_dst, proc)
Definition: lock.h:77
#define SetInvalidVirtualTransactionId(vxid)
Definition: lock.h:74
bool LWLockHeldByMe(LWLock *lock)
Definition: lwlock.c:1895
bool LWLockAcquire(LWLock *lock, LWLockMode mode)
Definition: lwlock.c:1170
bool LWLockHeldByMeInMode(LWLock *lock, LWLockMode mode)
Definition: lwlock.c:1939
void LWLockRelease(LWLock *lock)
Definition: lwlock.c:1783
void LWLockInitialize(LWLock *lock, int tranche_id)
Definition: lwlock.c:709
@ LWTRANCHE_SERIAL_SLRU
Definition: lwlock.h:214
@ LWTRANCHE_SERIAL_BUFFER
Definition: lwlock.h:185
@ LWTRANCHE_PER_XACT_PREDICATE_LIST
Definition: lwlock.h:202
@ LW_SHARED
Definition: lwlock.h:115
@ LW_EXCLUSIVE
Definition: lwlock.h:114
#define NUM_PREDICATELOCK_PARTITIONS
Definition: lwlock.h:101
void * palloc(Size size)
Definition: mcxt.c:1316
#define InvalidPid
Definition: miscadmin.h:32
const void size_t len
const void * data
static bool pg_lfind32(uint32 key, const uint32 *base, uint32 nelem)
Definition: pg_lfind.h:153
static rewind_source * source
Definition: pg_rewind.c:89
#define ERRCODE_T_R_SERIALIZATION_FAILURE
Definition: pgbench.c:76
#define InvalidOid
Definition: postgres_ext.h:36
unsigned int Oid
Definition: postgres_ext.h:31
void CheckPointPredicate(void)
Definition: predicate.c:1036
void PredicateLockPageSplit(Relation relation, BlockNumber oldblkno, BlockNumber newblkno)
Definition: predicate.c:3129
static void DecrementParentLocks(const PREDICATELOCKTARGETTAG *targettag)
Definition: predicate.c:2376
static HTAB * PredicateLockHash
Definition: predicate.c:398
static void SetPossibleUnsafeConflict(SERIALIZABLEXACT *roXact, SERIALIZABLEXACT *activeXact)
Definition: predicate.c:666
#define PredicateLockTargetTagHashCode(predicatelocktargettag)
Definition: predicate.c:303
static void SetNewSxactGlobalXmin(void)
Definition: predicate.c:3236
void PredicateLockTwoPhaseFinish(TransactionId xid, bool isCommit)
Definition: predicate.c:4867
PredicateLockData * GetPredicateLockStatusData(void)
Definition: predicate.c:1430
#define SerialPage(xid)
Definition: predicate.c:343
void InitPredicateLocks(void)
Definition: predicate.c:1140
static void ReleasePredXact(SERIALIZABLEXACT *sxact)
Definition: predicate.c:596
void SetSerializableTransactionSnapshot(Snapshot snapshot, VirtualTransactionId *sourcevxid, int sourcepid)
Definition: predicate.c:1707
static bool RWConflictExists(const SERIALIZABLEXACT *reader, const SERIALIZABLEXACT *writer)
Definition: predicate.c:610
static bool PredicateLockingNeededForRelation(Relation relation)
Definition: predicate.c:498
static bool SerializationNeededForRead(Relation relation, Snapshot snapshot)
Definition: predicate.c:516
static Snapshot GetSafeSnapshot(Snapshot origSnapshot)
Definition: predicate.c:1543
#define SxactIsCommitted(sxact)
Definition: predicate.c:277
static SerialControl serialControl
Definition: predicate.c:354
void PredicateLockPage(Relation relation, BlockNumber blkno, Snapshot snapshot)
Definition: predicate.c:2584
#define SxactIsROUnsafe(sxact)
Definition: predicate.c:292
static Snapshot GetSerializableTransactionSnapshotInt(Snapshot snapshot, VirtualTransactionId *sourcevxid, int sourcepid)
Definition: predicate.c:1749
static LWLock * ScratchPartitionLock
Definition: predicate.c:408
static void PredicateLockAcquire(const PREDICATELOCKTARGETTAG *targettag)
Definition: predicate.c:2502
#define SxactIsDeferrableWaiting(sxact)
Definition: predicate.c:290
static void ReleasePredicateLocksLocal(void)
Definition: predicate.c:3664
static HTAB * LocalPredicateLockHash
Definition: predicate.c:414
int max_predicate_locks_per_page
Definition: predicate.c:373
struct SerialControlData * SerialControl
Definition: predicate.c:352
static PredXactList PredXact
Definition: predicate.c:384
static void SetRWConflict(SERIALIZABLEXACT *reader, SERIALIZABLEXACT *writer)
Definition: predicate.c:643
int GetSafeSnapshotBlockingPids(int blocked_pid, int *output, int output_size)
Definition: predicate.c:1613
static uint32 ScratchTargetTagHash
Definition: predicate.c:407
static void RemoveTargetIfNoLongerUsed(PREDICATELOCKTARGET *target, uint32 targettaghash)
Definition: predicate.c:2168
static uint32 predicatelock_hash(const void *key, Size keysize)
Definition: predicate.c:1404
void CheckForSerializableConflictOut(Relation relation, TransactionId xid, Snapshot snapshot)
Definition: predicate.c:4008
#define SxactIsReadOnly(sxact)
Definition: predicate.c:281
#define SerialNextPage(page)
Definition: predicate.c:337
static void DropAllPredicateLocksFromTable(Relation relation, bool transfer)
Definition: predicate.c:2922
bool PageIsPredicateLocked(Relation relation, BlockNumber blkno)
Definition: predicate.c:1993
static SlruCtlData SerialSlruCtlData
Definition: predicate.c:324
static void CreatePredicateLock(const PREDICATELOCKTARGETTAG *targettag, uint32 targettaghash, SERIALIZABLEXACT *sxact)
Definition: predicate.c:2438
static void SerialAdd(TransactionId xid, SerCommitSeqNo minConflictCommitSeqNo)
Definition: predicate.c:858
static void ClearOldPredicateLocks(void)
Definition: predicate.c:3682
#define SxactHasSummaryConflictIn(sxact)
Definition: predicate.c:282
static SERIALIZABLEXACT * CreatePredXact(void)
Definition: predicate.c:582
static bool GetParentPredicateLockTag(const PREDICATELOCKTARGETTAG *tag, PREDICATELOCKTARGETTAG *parent)
Definition: predicate.c:2057
#define PredicateLockHashCodeFromTargetHashCode(predicatelocktag, targethash)
Definition: predicate.c:316
static void RestoreScratchTarget(bool lockheld)
Definition: predicate.c:2146
#define SerialValue(slotno, xid)
Definition: predicate.c:339
static void DeleteChildTargetLocks(const PREDICATELOCKTARGETTAG *newtargettag)
Definition: predicate.c:2199
static void DeleteLockTarget(PREDICATELOCKTARGET *target, uint32 targettaghash)
Definition: predicate.c:2654
static SERIALIZABLEXACT * OldCommittedSxact
Definition: predicate.c:362
#define SxactHasConflictOut(sxact)
Definition: predicate.c:289
void CheckForSerializableConflictIn(Relation relation, ItemPointer tid, BlockNumber blkno)
Definition: predicate.c:4321
static bool MyXactDidWrite
Definition: predicate.c:422
static int MaxPredicateChildLocks(const PREDICATELOCKTARGETTAG *tag)
Definition: predicate.c:2274
static void FlagSxactUnsafe(SERIALIZABLEXACT *sxact)
Definition: predicate.c:699
static void SerialInit(void)
Definition: predicate.c:806
void CheckTableForSerializableConflictIn(Relation relation)
Definition: predicate.c:4404
#define SxactIsPrepared(sxact)
Definition: predicate.c:278
void PredicateLockTID(Relation relation, ItemPointer tid, Snapshot snapshot, TransactionId tuple_xid)
Definition: predicate.c:2606
void AttachSerializableXact(SerializableXactHandle handle)
Definition: predicate.c:5040
struct SerialControlData SerialControlData
SerializableXactHandle ShareSerializableXact(void)
Definition: predicate.c:5031
static bool PredicateLockExists(const PREDICATELOCKTARGETTAG *targettag)
Definition: predicate.c:2030
static void RemoveScratchTarget(bool lockheld)
Definition: predicate.c:2125
#define SxactIsOnFinishedList(sxact)
Definition: predicate.c:267
#define SxactIsPartiallyReleased(sxact)
Definition: predicate.c:293
static void SerialSetActiveSerXmin(TransactionId xid)
Definition: predicate.c:985
static dlist_head * FinishedSerializableTransactions
Definition: predicate.c:399
static bool SerializationNeededForWrite(Relation relation)
Definition: predicate.c:560
static HTAB * SerializableXidHash
Definition: predicate.c:396
static bool CheckAndPromotePredicateLockRequest(const PREDICATELOCKTARGETTAG *reqtag)
Definition: predicate.c:2311
void PredicateLockPageCombine(Relation relation, BlockNumber oldblkno, BlockNumber newblkno)
Definition: predicate.c:3214
static bool SerialPagePrecedesLogically(int64 page1, int64 page2)
Definition: predicate.c:731
static void CheckTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag)
Definition: predicate.c:4151
int max_predicate_locks_per_relation
Definition: predicate.c:372
#define SxactIsROSafe(sxact)
Definition: predicate.c:291
void PreCommit_CheckForSerializationFailure(void)
Definition: predicate.c:4688
void ReleasePredicateLocks(bool isCommit, bool isReadOnlySafe)
Definition: predicate.c:3297
static void FlagRWConflict(SERIALIZABLEXACT *reader, SERIALIZABLEXACT *writer)
Definition: predicate.c:4486
static const PREDICATELOCKTARGETTAG ScratchTargetTag
Definition: predicate.c:406
#define PredicateLockHashPartitionLockByIndex(i)
Definition: predicate.c:261
static void OnConflict_CheckForSerializationFailure(const SERIALIZABLEXACT *reader, SERIALIZABLEXACT *writer)
Definition: predicate.c:4521
static bool CoarserLockCovers(const PREDICATELOCKTARGETTAG *newtargettag)
Definition: predicate.c:2096
void PredicateLockRelation(Relation relation, Snapshot snapshot)
Definition: predicate.c:2561
static SERIALIZABLEXACT * MySerializableXact
Definition: predicate.c:421
void predicatelock_twophase_recover(TransactionId xid, uint16 info, void *recdata, uint32 len)
Definition: predicate.c:4894
Size PredicateLockShmemSize(void)
Definition: predicate.c:1342
#define SxactIsDoomed(sxact)
Definition: predicate.c:280
#define NPREDICATELOCKTARGETENTS()
Definition: predicate.c:264
static SerCommitSeqNo SerialGetMinConflictCommitSeqNo(TransactionId xid)
Definition: predicate.c:944
static void SummarizeOldestCommittedSxact(void)
Definition: predicate.c:1488
bool check_serial_buffers(int *newval, void **extra, GucSource source)
Definition: predicate.c:847
#define TargetTagIsCoveredBy(covered_target, covering_target)
Definition: predicate.c:233
static RWConflictPoolHeader RWConflictPool
Definition: predicate.c:390
static void ReleaseRWConflict(RWConflict conflict)
Definition: predicate.c:691
static bool TransferPredicateLocksToNewTarget(PREDICATELOCKTARGETTAG oldtargettag, PREDICATELOCKTARGETTAG newtargettag, bool removeOld)
Definition: predicate.c:2715
void AtPrepare_PredicateLocks(void)
Definition: predicate.c:4775
void RegisterPredicateLockingXid(TransactionId xid)
Definition: predicate.c:1944
#define PredicateLockHashPartitionLock(hashcode)
Definition: predicate.c:258
#define SERIAL_ENTRIESPERPAGE
Definition: predicate.c:330
static bool XidIsConcurrent(TransactionId xid)
Definition: predicate.c:3957
static void ReleaseOneSerializableXact(SERIALIZABLEXACT *sxact, bool partial, bool summarize)
Definition: predicate.c:3820
static HTAB * PredicateLockTargetHash
Definition: predicate.c:397
bool CheckForSerializableConflictOutNeeded(Relation relation, Snapshot snapshot)
Definition: predicate.c:3976
#define SxactIsRolledBack(sxact)
Definition: predicate.c:279
static SERIALIZABLEXACT * SavedSerializableXact
Definition: predicate.c:431
#define SxactHasSummaryConflictOut(sxact)
Definition: predicate.c:283
void TransferPredicateLocksToHeapRelation(Relation relation)
Definition: predicate.c:3108
void PostPrepare_PredicateLocks(TransactionId xid)
Definition: predicate.c:4844
static void CreateLocalPredicateLockHash(void)
Definition: predicate.c:1925
#define SerialSlruCtl
Definition: predicate.c:326
int max_predicate_locks_per_xact
Definition: predicate.c:371
Snapshot GetSerializableTransactionSnapshot(Snapshot snapshot)
Definition: predicate.c:1667
void * SerializableXactHandle
Definition: predicate.h:33
#define RWConflictDataSize
#define SXACT_FLAG_DEFERRABLE_WAITING
#define SXACT_FLAG_SUMMARY_CONFLICT_IN
@ TWOPHASEPREDICATERECORD_XACT
@ TWOPHASEPREDICATERECORD_LOCK
#define FirstNormalSerCommitSeqNo
#define InvalidSerCommitSeqNo
@ PREDLOCKTAG_RELATION
@ PREDLOCKTAG_PAGE
@ PREDLOCKTAG_TUPLE
struct PREDICATELOCKTAG PREDICATELOCKTAG
#define SXACT_FLAG_CONFLICT_OUT
#define PredXactListDataSize
#define SXACT_FLAG_READ_ONLY
#define SXACT_FLAG_DOOMED
struct LOCALPREDICATELOCK LOCALPREDICATELOCK
#define GET_PREDICATELOCKTARGETTAG_DB(locktag)
#define GET_PREDICATELOCKTARGETTAG_RELATION(locktag)
#define RWConflictPoolHeaderDataSize
struct SERIALIZABLEXIDTAG SERIALIZABLEXIDTAG
#define InvalidSerializableXact
struct PREDICATELOCKTARGET PREDICATELOCKTARGET
#define SET_PREDICATELOCKTARGETTAG_PAGE(locktag, dboid, reloid, blocknum)
#define RecoverySerCommitSeqNo
struct PREDICATELOCKTARGETTAG PREDICATELOCKTARGETTAG
struct SERIALIZABLEXID SERIALIZABLEXID
#define GET_PREDICATELOCKTARGETTAG_TYPE(locktag)
#define SET_PREDICATELOCKTARGETTAG_RELATION(locktag, dboid, reloid)
uint64 SerCommitSeqNo
#define SXACT_FLAG_ROLLED_BACK
#define SXACT_FLAG_COMMITTED
#define SXACT_FLAG_RO_UNSAFE
#define SXACT_FLAG_PREPARED
#define SET_PREDICATELOCKTARGETTAG_TUPLE(locktag, dboid, reloid, blocknum, offnum)
#define SXACT_FLAG_PARTIALLY_RELEASED
#define GET_PREDICATELOCKTARGETTAG_PAGE(locktag)
#define SXACT_FLAG_RO_SAFE
struct PREDICATELOCK PREDICATELOCK
#define SXACT_FLAG_SUMMARY_CONFLICT_OUT
#define GET_PREDICATELOCKTARGETTAG_OFFSET(locktag)
Snapshot GetSnapshotData(Snapshot snapshot)
Definition: procarray.c:2165
bool ProcArrayInstallImportedXmin(TransactionId xmin, VirtualTransactionId *sourcevxid)
Definition: procarray.c:2524
#define INVALID_PROC_NUMBER
Definition: procnumber.h:26
#define RelationUsesLocalBuffers(relation)
Definition: rel.h:637
bool ShmemAddrIsValid(const void *addr)
Definition: shmem.c:274
void * ShmemAlloc(Size size)
Definition: shmem.c:152
Size add_size(Size s1, Size s2)
Definition: shmem.c:493
void * ShmemInitStruct(const char *name, Size size, bool *foundPtr)
Definition: shmem.c:387
Size mul_size(Size s1, Size s2)
Definition: shmem.c:510
HTAB * ShmemInitHash(const char *name, long init_size, long max_size, HASHCTL *infoP, int hash_flags)
Definition: shmem.c:332
static pg_noinline void Size size
Definition: slab.c:607
void SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns, const char *subdir, int buffer_tranche_id, int bank_tranche_id, SyncRequestHandler sync_handler, bool long_segment_names)
Definition: slru.c:238
int SimpleLruReadPage_ReadOnly(SlruCtl ctl, int64 pageno, TransactionId xid)
Definition: slru.c:591
void SimpleLruWriteAll(SlruCtl ctl, bool allow_redirtied)
Definition: slru.c:1305
int SimpleLruReadPage(SlruCtl ctl, int64 pageno, bool write_ok, TransactionId xid)
Definition: slru.c:488
int SimpleLruZeroPage(SlruCtl ctl, int64 pageno)
Definition: slru.c:361
void SimpleLruTruncate(SlruCtl ctl, int64 cutoffPage)
Definition: slru.c:1391
Size SimpleLruShmemSize(int nslots, int nlsns)
Definition: slru.c:184
bool check_slru_buffers(const char *name, int *newval)
Definition: slru.c:341
static LWLock * SimpleLruGetBankLock(SlruCtl ctl, int64 pageno)
Definition: slru.h:179
#define SlruPagePrecedesUnitTests(ctl, per_page)
Definition: slru.h:203
#define SLRU_PAGES_PER_SEGMENT
Definition: slru.h:39
Snapshot GetTransactionSnapshot(void)
Definition: snapmgr.c:216
#define IsMVCCSnapshot(snapshot)
Definition: snapmgr.h:62
void ProcSendSignal(ProcNumber procNumber)
Definition: proc.c:1878
PGPROC * MyProc
Definition: proc.c:66
void ProcWaitForSignal(uint32 wait_event_info)
Definition: proc.c:1866
Size keysize
Definition: hsearch.h:75
HashValueFunc hash
Definition: hsearch.h:78
Size entrysize
Definition: hsearch.h:76
long num_partitions
Definition: hsearch.h:68
Definition: dynahash.c:220
Definition: lwlock.h:42
Definition: proc.h:157
SERIALIZABLEXACT * myXact
PREDICATELOCKTARGET * myTarget
PREDICATELOCKTARGETTAG tag
PREDICATELOCKTAG tag
SerCommitSeqNo commitSeqNo
SERIALIZABLEXACT * element
SerCommitSeqNo LastSxactCommitSeqNo
SerCommitSeqNo CanPartialClearThrough
SERIALIZABLEXACT * OldCommittedSxact
SerCommitSeqNo HavePartialClearedThrough
TransactionId SxactGlobalXmin
SERIALIZABLEXACT * sxactIn
SERIALIZABLEXACT * sxactOut
Form_pg_index rd_index
Definition: rel.h:192
Oid rd_id
Definition: rel.h:113
RelFileLocator rd_locator
Definition: rel.h:57
VirtualTransactionId vxid
union SERIALIZABLEXACT::@115 SeqNo
SerCommitSeqNo lastCommitBeforeSnapshot
dlist_head possibleUnsafeConflicts
SerCommitSeqNo prepareSeqNo
SerCommitSeqNo commitSeqNo
TransactionId finishedBefore
SerCommitSeqNo earliestOutConflictCommit
SERIALIZABLEXACT * myXact
TransactionId headXid
Definition: predicate.c:348
TransactionId tailXid
Definition: predicate.c:349
TransactionId xmin
Definition: snapshot.h:157
uint32 xcnt
Definition: snapshot.h:169
TransactionId xmax
Definition: snapshot.h:158
TransactionId * xip
Definition: snapshot.h:168
FullTransactionId nextXid
Definition: transam.h:220
PREDICATELOCKTARGETTAG target
TwoPhasePredicateRecordType type
union TwoPhasePredicateRecord::@116 data
TwoPhasePredicateLockRecord lockRecord
TwoPhasePredicateXactRecord xactRecord
LocalTransactionId localTransactionId
Definition: lock.h:62
ProcNumber procNumber
Definition: lock.h:61
dlist_node * cur
Definition: ilist.h:179
dlist_node * cur
Definition: ilist.h:200
@ SYNC_HANDLER_NONE
Definition: sync.h:42
bool TransactionIdPrecedes(TransactionId id1, TransactionId id2)
Definition: transam.c:280
bool TransactionIdPrecedesOrEquals(TransactionId id1, TransactionId id2)
Definition: transam.c:299
bool TransactionIdFollows(TransactionId id1, TransactionId id2)
Definition: transam.c:314
bool TransactionIdFollowsOrEquals(TransactionId id1, TransactionId id2)
Definition: transam.c:329
#define FirstUnpinnedObjectId
Definition: transam.h:196
#define InvalidTransactionId
Definition: transam.h:31
#define TransactionIdEquals(id1, id2)
Definition: transam.h:43
#define XidFromFullTransactionId(x)
Definition: transam.h:48
#define FirstNormalTransactionId
Definition: transam.h:34
#define TransactionIdIsValid(xid)
Definition: transam.h:41
void RegisterTwoPhaseRecord(TwoPhaseRmgrId rmid, uint16 info, const void *data, uint32 len)
Definition: twophase.c:1280
int max_prepared_xacts
Definition: twophase.c:115
#define TWOPHASE_RM_PREDICATELOCK_ID
Definition: twophase_rmgr.h:28
TransamVariablesData * TransamVariables
Definition: varsup.c:34
bool XactDeferrable
Definition: xact.c:83
bool XactReadOnly
Definition: xact.c:80
TransactionId GetTopTransactionIdIfAny(void)
Definition: xact.c:438
bool IsSubTransaction(void)
Definition: xact.c:4988
bool TransactionIdIsCurrentTransactionId(TransactionId xid)
Definition: xact.c:938
bool IsInParallelMode(void)
Definition: xact.c:1086
#define IsolationIsSerializable()
Definition: xact.h:52
bool RecoveryInProgress(void)
Definition: xlog.c:6290