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