src/backend/storage/lmgr/README Locking Overview ================ Postgres uses four types of interprocess locks: * Spinlocks. These are intended for *very* short-term locks. If a lock is to be held more than a few dozen instructions, or across any sort of kernel call (or even a call to a nontrivial subroutine), don't use a spinlock. Spinlocks are primarily used as infrastructure for lightweight locks. They are implemented using a hardware atomic-test-and-set instruction, if available. Waiting processes busy-loop until they can get the lock. There is no provision for deadlock detection, automatic release on error, or any other nicety. There is a timeout if the lock cannot be gotten after a minute or so (which is approximately forever in comparison to the intended lock hold time, so this is certainly an error condition). * Lightweight locks (LWLocks). These locks are typically used to interlock access to datastructures in shared memory. LWLocks support both exclusive and shared lock modes (for read/write and read-only access to a shared object). There is no provision for deadlock detection, but the LWLock manager will automatically release held LWLocks during elog() recovery, so it is safe to raise an error while holding LWLocks. Obtaining or releasing an LWLock is quite fast (a few dozen instructions) when there is no contention for the lock. When a process has to wait for an LWLock, it blocks on a SysV semaphore so as to not consume CPU time. Waiting processes will be granted the lock in arrival order. There is no timeout. * Regular locks (a/k/a heavyweight locks). The regular lock manager supports a variety of lock modes with table-driven semantics, and it has full deadlock detection and automatic release at transaction end. Regular locks should be used for all user-driven lock requests. * SIReadLock predicate locks. See separate README-SSI file for details. Acquisition of either a spinlock or a lightweight lock causes query cancel and die() interrupts to be held off until all such locks are released. No such restriction exists for regular locks, however. Also note that we can accept query cancel and die() interrupts while waiting for a regular lock, but we will not accept them while waiting for spinlocks or LW locks. It is therefore not a good idea to use LW locks when the wait time might exceed a few seconds. The rest of this README file discusses the regular lock manager in detail. Lock Data Structures -------------------- Lock methods describe the overall locking behavior. Currently there are two lock methods: DEFAULT and USER. Lock modes describe the type of the lock (read/write or shared/exclusive). In principle, each lock method can have its own set of lock modes with different conflict rules, but currently DEFAULT and USER methods use identical lock mode sets. See src/include/storage/lock.h for more details. (Lock modes are also called lock types in some places in the code and documentation.) There are two main methods for recording locks in shared memory. The primary mechanism uses two main structures: the per-lockable-object LOCK struct, and the per-lock-and-requestor PROCLOCK struct. A LOCK object exists for each lockable object that currently has locks held or requested on it. A PROCLOCK struct exists for each backend that is holding or requesting lock(s) on each LOCK object. There is also a special "fast path" mechanism which backends may use to record a limited number of locks with very specific characteristics: they must use the DEFAULT lockmethod; they must represent a lock on a database relation (not a shared relation), they must be a "weak" lock which is unlikely to conflict (AccessShareLock, RowShareLock, or RowExclusiveLock); and the system must be able to quickly verify that no conflicting locks could possibly be present. See "Fast Path Locking", below, for more details. Each backend also maintains an unshared LOCALLOCK structure for each lockable object and lock mode that it is currently holding or requesting. The shared lock structures only allow a single lock grant to be made per lockable object/lock mode/backend. Internally to a backend, however, the same lock may be requested and perhaps released multiple times in a transaction, and it can also be held both transactionally and session-wide. The internal request counts are held in LOCALLOCK so that the shared data structures need not be accessed to alter them. --------------------------------------------------------------------------- The lock manager's LOCK objects contain: tag - The key fields that are used for hashing locks in the shared memory lock hash table. The contents of the tag essentially define an individual lockable object. See include/storage/lock.h for details about the supported types of lockable objects. This is declared as a separate struct to ensure that we always zero out the correct number of bytes. It is critical that any alignment-padding bytes the compiler might insert in the struct be zeroed out, else the hash computation will be random. (Currently, we are careful to define struct LOCKTAG so that there are no padding bytes.) grantMask - This bitmask indicates what types of locks are currently held on the given lockable object. It is used (against the lock table's conflict table) to determine if a new lock request will conflict with existing lock types held. Conflicts are determined by bitwise AND operations between the grantMask and the conflict table entry for the requested lock type. Bit i of grantMask is 1 if and only if granted[i] > 0. waitMask - This bitmask shows the types of locks being waited for. Bit i of waitMask is 1 if and only if requested[i] > granted[i]. procLocks - This is a shared memory queue of all the PROCLOCK structs associated with the lock object. Note that both granted and waiting PROCLOCKs are in this list (indeed, the same PROCLOCK might have some already-granted locks and be waiting for more!). waitProcs - This is a shared memory queue of all PGPROC structures corresponding to backends that are waiting (sleeping) until another backend releases this lock. The process structure holds the information needed to determine if it should be woken up when the lock is released. nRequested - Keeps a count of how many times this lock has been attempted to be acquired. The count includes attempts by processes which were put to sleep due to conflicts. It also counts the same backend twice if, for example, a backend process first acquires a read and then acquires a write. (But multiple acquisitions of the same lock/lock mode within a backend are not multiply counted here; they are recorded only in the backend's LOCALLOCK structure.) requested - Keeps a count of how many locks of each type have been attempted. Only elements 1 through MAX_LOCKMODES-1 are used as they correspond to the lock type defined constants. Summing the values of requested[] should come out equal to nRequested. nGranted - Keeps count of how many times this lock has been successfully acquired. This count does not include attempts that are waiting due to conflicts. Otherwise the counting rules are the same as for nRequested. granted - Keeps count of how many locks of each type are currently held. Once again only elements 1 through MAX_LOCKMODES-1 are used (0 is not). Also, like requested[], summing the values of granted[] should total to the value of nGranted. We should always have 0 <= nGranted <= nRequested, and 0 <= granted[i] <= requested[i] for each i. When all the request counts go to zero, the LOCK object is no longer needed and can be freed. --------------------------------------------------------------------------- The lock manager's PROCLOCK objects contain: tag - The key fields that are used for hashing entries in the shared memory PROCLOCK hash table. This is declared as a separate struct to ensure that we always zero out the correct number of bytes. It is critical that any alignment-padding bytes the compiler might insert in the struct be zeroed out, else the hash computation will be random. (Currently, we are careful to define struct PROCLOCKTAG so that there are no padding bytes.) tag.myLock Pointer to the shared LOCK object this PROCLOCK is for. tag.myProc Pointer to the PGPROC of backend process that owns this PROCLOCK. Note: it's OK to use pointers here because a PROCLOCK never outlives either its lock or its proc. The tag is therefore unique for as long as it needs to be, even though the same tag values might mean something else at other times. holdMask - A bitmask for the lock modes successfully acquired by this PROCLOCK. This should be a subset of the LOCK object's grantMask, and also a subset of the PGPROC object's heldLocks mask (if the PGPROC is currently waiting for another lock mode on this lock). releaseMask - A bitmask for the lock modes due to be released during LockReleaseAll. This must be a subset of the holdMask. Note that it is modified without taking the partition LWLock, and therefore it is unsafe for any backend except the one owning the PROCLOCK to examine/change it. lockLink - List link for shared memory queue of all the PROCLOCK objects for the same LOCK. procLink - List link for shared memory queue of all the PROCLOCK objects for the same backend. --------------------------------------------------------------------------- Lock Manager Internal Locking ----------------------------- Before PostgreSQL 8.2, all of the shared-memory data structures used by the lock manager were protected by a single LWLock, the LockMgrLock; any operation involving these data structures had to exclusively lock LockMgrLock. Not too surprisingly, this became a contention bottleneck. To reduce contention, the lock manager's data structures have been split into multiple "partitions", each protected by an independent LWLock. Most operations only need to lock the single partition they are working in. Here are the details: * Each possible lock is assigned to one partition according to a hash of its LOCKTAG value. The partition's LWLock is considered to protect all the LOCK objects of that partition as well as their subsidiary PROCLOCKs. * The shared-memory hash tables for LOCKs and PROCLOCKs are organized so that different partitions use different hash chains, and thus there is no conflict in working with objects in different partitions. This is supported directly by dynahash.c's "partitioned table" mechanism for the LOCK table: we need only ensure that the partition number is taken from the low-order bits of the dynahash hash value for the LOCKTAG. To make it work for PROCLOCKs, we have to ensure that a PROCLOCK's hash value has the same low-order bits as its associated LOCK. This requires a specialized hash function (see proclock_hash). * Formerly, each PGPROC had a single list of PROCLOCKs belonging to it. This has now been split into per-partition lists, so that access to a particular PROCLOCK list can be protected by the associated partition's LWLock. (This rule allows one backend to manipulate another backend's PROCLOCK lists, which was not originally necessary but is now required in connection with fast-path locking; see below.) * The other lock-related fields of a PGPROC are only interesting when the PGPROC is waiting for a lock, so we consider that they are protected by the partition LWLock of the awaited lock. For normal lock acquisition and release, it is sufficient to lock the partition containing the desired lock. Deadlock checking needs to touch multiple partitions in general; for simplicity, we just make it lock all the partitions in partition-number order. (To prevent LWLock deadlock, we establish the rule that any backend needing to lock more than one partition at once must lock them in partition-number order.) It's possible that deadlock checking could be done without touching every partition in typical cases, but since in a properly functioning system deadlock checking should not occur often enough to be performance-critical, trying to make this work does not seem a productive use of effort. A backend's internal LOCALLOCK hash table is not partitioned. We do store a copy of the locktag hash code in LOCALLOCK table entries, from which the partition number can be computed, but this is a straight speed-for-space tradeoff: we could instead recalculate the partition number from the LOCKTAG when needed. Fast Path Locking ----------------- Fast path locking is a special purpose mechanism designed to reduce the overhead of taking and releasing certain types of locks which are taken and released very frequently but rarely conflict. Currently, this includes two categories of locks: (1) Weak relation locks. SELECT, INSERT, UPDATE, and DELETE must acquire a lock on every relation they operate on, as well as various system catalogs that can be used internally. Many DML operations can proceed in parallel against the same table at the same time; only DDL operations such as CLUSTER, ALTER TABLE, or DROP -- or explicit user action such as LOCK TABLE -- will create lock conflicts with the "weak" locks (AccessShareLock, RowShareLock, RowExclusiveLock) acquired by DML operations. (2) VXID locks. Every transaction takes a lock on its own virtual transaction ID. Currently, the only operations that wait for these locks are CREATE INDEX CONCURRENTLY and Hot Standby (in the case of a conflict), so most VXID locks are taken and released by the owner without anyone else needing to care. The primary locking mechanism does not cope well with this workload. Even though the lock manager locks are partitioned, the locktag for any given relation still falls in one, and only one, partition. Thus, if many short queries are accessing the same relation, the lock manager partition lock for that partition becomes a contention bottleneck. This effect is measurable even on 2-core servers, and becomes very pronounced as core count increases. To alleviate this bottleneck, beginning in PostgreSQL 9.2, each backend is permitted to record a limited number of locks on unshared relations in an array within its PGPROC structure, rather than using the primary lock table. This mechanism can only be used when the locker can verify that no conflicting locks exist at the time of taking the lock. A key point of this algorithm is that it must be possible to verify the absence of possibly conflicting locks without fighting over a shared LWLock or spinlock. Otherwise, this effort would simply move the contention bottleneck from one place to another. We accomplish this using an array of 1024 integer counters, which are in effect a 1024-way partitioning of the lock space. Each counter records the number of "strong" locks (that is, ShareLock, ShareRowExclusiveLock, ExclusiveLock, and AccessExclusiveLock) on unshared relations that fall into that partition. When this counter is non-zero, the fast path mechanism may not be used to take new relation locks within that partition. A strong locker bumps the counter and then scans each per-backend array for matching fast-path locks; any which are found must be transferred to the primary lock table before attempting to acquire the lock, to ensure proper lock conflict and deadlock detection. On an SMP system, we must guarantee proper memory synchronization. Here we rely on the fact that LWLock acquisition acts as a memory sequence point: if A performs a store, A and B both acquire an LWLock in either order, and B then performs a load on the same memory location, it is guaranteed to see A's store. In this case, each backend's fast-path lock queue is protected by an LWLock. A backend wishing to acquire a fast-path lock grabs this LWLock before examining FastPathStrongRelationLocks to check for the presence of a conflicting strong lock. And the backend attempting to acquire a strong lock, because it must transfer any matching weak locks taken via the fast-path mechanism to the shared lock table, will acquire every LWLock protecting a backend fast-path queue in turn. So, if we examine FastPathStrongRelationLocks and see a zero, then either the value is truly zero, or if it is a stale value, the strong locker has yet to acquire the per-backend LWLock we now hold (or, indeed, even the first per-backend LWLock) and will notice any weak lock we take when it does. Fast-path VXID locks do not use the FastPathStrongRelationLocks table. The first lock taken on a VXID is always the ExclusiveLock taken by its owner. Any subsequent lockers are share lockers waiting for the VXID to terminate. Indeed, the only reason VXID locks use the lock manager at all (rather than waiting for the VXID to terminate via some other method) is for deadlock detection. Thus, the initial VXID lock can *always* be taken via the fast path without checking for conflicts. Any subsequent locker must check whether the lock has been transferred to the main lock table, and if not, do so. The backend owning the VXID must be careful to clean up any entry made in the main lock table at end of transaction. Deadlock detection does not need to examine the fast-path data structures, because any lock that could possibly be involved in a deadlock must have been transferred to the main tables beforehand. The Deadlock Detection Algorithm -------------------------------- Since we allow user transactions to request locks in any order, deadlock is possible. We use a deadlock detection/breaking algorithm that is fairly standard in essence, but there are many special considerations needed to deal with Postgres' generalized locking model. A key design consideration is that we want to make routine operations (lock grant and release) run quickly when there is no deadlock, and avoid the overhead of deadlock handling as much as possible. We do this using an "optimistic waiting" approach: if a process cannot acquire the lock it wants immediately, it goes to sleep without any deadlock check. But it also sets a delay timer, with a delay of DeadlockTimeout milliseconds (typically set to one second). If the delay expires before the process is granted the lock it wants, it runs the deadlock detection/breaking code. Normally this code will determine that there is no deadlock condition, and then the process will go back to sleep and wait quietly until it is granted the lock. But if a deadlock condition does exist, it will be resolved, usually by aborting the detecting process' transaction. In this way, we avoid deadlock handling overhead whenever the wait time for a lock is less than DeadlockTimeout, while not imposing an unreasonable delay of detection when there is an error. Lock acquisition (routines LockAcquire and ProcSleep) follows these rules: 1. A lock request is granted immediately if it does not conflict with any existing or waiting lock request, or if the process already holds an instance of the same lock type (eg, there's no penalty to acquire a read lock twice). Note that a process never conflicts with itself, eg one can obtain read lock when one already holds exclusive lock. 2. Otherwise the process joins the lock's wait queue. Normally it will be added to the end of the queue, but there is an exception: if the process already holds locks on this same lockable object that conflict with the request of any pending waiter, then the process will be inserted in the wait queue just ahead of the first such waiter. (If we did not make this check, the deadlock detection code would adjust the queue order to resolve the conflict, but it's relatively cheap to make the check in ProcSleep and avoid a deadlock timeout delay in this case.) Note special case when inserting before the end of the queue: if the process's request does not conflict with any existing lock nor any waiting request before its insertion point, then go ahead and grant the lock without waiting. When a lock is released, the lock release routine (ProcLockWakeup) scans the lock object's wait queue. Each waiter is awoken if (a) its request does not conflict with already-granted locks, and (b) its request does not conflict with the requests of prior un-wakable waiters. Rule (b) ensures that conflicting requests are granted in order of arrival. There are cases where a later waiter must be allowed to go in front of conflicting earlier waiters to avoid deadlock, but it is not ProcLockWakeup's responsibility to recognize these cases; instead, the deadlock detection code will re-order the wait queue when necessary. To perform deadlock checking, we use the standard method of viewing the various processes as nodes in a directed graph (the waits-for graph or WFG). There is a graph edge leading from process A to process B if A waits for B, ie, A is waiting for some lock and B holds a conflicting lock. There is a deadlock condition if and only if the WFG contains a cycle. We detect cycles by searching outward along waits-for edges to see if we return to our starting point. There are three possible outcomes: 1. All outgoing paths terminate at a running process (which has no outgoing edge). 2. A deadlock is detected by looping back to the start point. We resolve such a deadlock by canceling the start point's lock request and reporting an error in that transaction, which normally leads to transaction abort and release of that transaction's held locks. Note that it's sufficient to cancel one request to remove the cycle; we don't need to kill all the transactions involved. 3. Some path(s) loop back to a node other than the start point. This indicates a deadlock, but one that does not involve our starting process. We ignore this condition on the grounds that resolving such a deadlock is the responsibility of the processes involved --- killing our start-point process would not resolve the deadlock. So, cases 1 and 3 both report "no deadlock". Postgres' situation is a little more complex than the standard discussion of deadlock detection, for two reasons: 1. A process can be waiting for more than one other process, since there might be multiple PROCLOCKs of (non-conflicting) lock types that all conflict with the waiter's request. This creates no real difficulty however; we simply need to be prepared to trace more than one outgoing edge. 2. If a process A is behind a process B in some lock's wait queue, and their requested locks conflict, then we must say that A waits for B, since ProcLockWakeup will never awaken A before B. This creates additional edges in the WFG. We call these "soft" edges, as opposed to the "hard" edges induced by locks already held. Note that if B already holds any locks conflicting with A's request, then their relationship is a hard edge not a soft edge. A "soft" block, or wait-priority block, has the same potential for inducing deadlock as a hard block. However, we may be able to resolve a soft block without aborting the transactions involved: we can instead rearrange the order of the wait queue. This rearrangement reverses the direction of the soft edge between two processes with conflicting requests whose queue order is reversed. If we can find a rearrangement that eliminates a cycle without creating new ones, then we can avoid an abort. Checking for such possible rearrangements is the trickiest part of the algorithm. The workhorse of the deadlock detector is a routine FindLockCycle() which is given a starting point process (which must be a waiting process). It recursively scans outward across waits-for edges as discussed above. If it finds no cycle involving the start point, it returns "false". (As discussed above, we can ignore cycles not involving the start point.) When such a cycle is found, FindLockCycle() returns "true", and as it unwinds it also builds a list of any "soft" edges involved in the cycle. If the resulting list is empty then there is a hard deadlock and the configuration cannot succeed. However, if the list is not empty, then reversing any one of the listed edges through wait-queue rearrangement will eliminate that cycle. Since such a reversal might create cycles elsewhere, we may need to try every possibility. Therefore, we need to be able to invoke FindLockCycle() on hypothetical configurations (wait orders) as well as the current real order. The easiest way to handle this seems to be to have a lookaside table that shows the proposed new queue order for each wait queue that we are considering rearranging. This table is checked by FindLockCycle, and it believes the proposed queue order rather than the real order for each lock that has an entry in the lookaside table. We build a proposed new queue order by doing a "topological sort" of the existing entries. Each soft edge that we are currently considering reversing creates a property of the partial order that the topological sort has to enforce. We must use a sort method that preserves the input ordering as much as possible, so as not to gratuitously break arrival order for processes not involved in a deadlock. (This is not true of the tsort method shown in Knuth, for example, but it's easily done by a simple doubly-nested-loop method that emits the first legal candidate at each step. Fortunately, we don't need a highly efficient sort algorithm, since the number of partial order constraints is not likely to be large.) Note that failure of the topological sort tells us we have conflicting ordering constraints, and therefore that the last-added soft edge reversal conflicts with a prior edge reversal. We need to detect this case to avoid an infinite loop in the case where no possible rearrangement will work: otherwise, we might try a reversal, find that it still leads to a cycle, then try to un-reverse the reversal while trying to get rid of that cycle, etc etc. Topological sort failure tells us the un-reversal is not a legitimate move in this context. So, the basic step in our rearrangement method is to take a list of soft edges in a cycle (as returned by FindLockCycle()) and successively try the reversal of each one as a topological-sort constraint added to whatever constraints we are already considering. We recursively search through all such sets of constraints to see if any one eliminates all the deadlock cycles at once. Although this might seem impossibly inefficient, it shouldn't be a big problem in practice, because there will normally be very few, and not very large, deadlock cycles --- if any at all. So the combinatorial inefficiency isn't going to hurt us. Besides, it's better to spend some time to guarantee that we've checked all possible escape routes than to abort a transaction when we didn't really have to. Each edge reversal constraint can be viewed as requesting that the waiting process A be moved to before the blocking process B in the wait queue they are both in. This action will reverse the desired soft edge, as well as any other soft edges between A and other processes it is advanced over. No other edges will be affected (note this is actually a constraint on our topological sort method to not re-order the queue more than necessary.) Therefore, we can be sure we have not created any new deadlock cycles if neither FindLockCycle(A) nor FindLockCycle(B) discovers any cycle. Given the above-defined behavior of FindLockCycle, each of these searches is necessary as well as sufficient, since FindLockCycle starting at the original start point will not complain about cycles that include A or B but not the original start point. In short then, a proposed rearrangement of the wait queue(s) is determined by one or more broken soft edges A->B, fully specified by the output of topological sorts of each wait queue involved, and then tested by invoking FindLockCycle() starting at the original start point as well as each of the mentioned processes (A's and B's). If none of the tests detect a cycle, then we have a valid configuration and can implement it by reordering the wait queues per the sort outputs (and then applying ProcLockWakeup on each reordered queue, in case a waiter has become wakable). If any test detects a soft cycle, we can try to resolve it by adding each soft link in that cycle, in turn, to the proposed rearrangement list. This is repeated recursively until we either find a workable rearrangement or determine that none exists. In the latter case, the outer level resolves the deadlock by aborting the original start-point transaction. The particular order in which rearrangements are tried depends on the order FindLockCycle() happens to scan in, so if there are multiple workable rearrangements of the wait queues, then it is unspecified which one will be chosen. What's more important is that we guarantee to try every queue rearrangement that could lead to success. (For example, if we have A before B before C and the needed order constraints are C before A and B before C, we would first discover that A before C doesn't work and try the rearrangement C before A before B. This would eventually lead to the discovery of the additional constraint B before C.) Got that? Miscellaneous Notes ------------------- 1. It is easily proven that no deadlock will be missed due to our asynchronous invocation of deadlock checking. A deadlock cycle in the WFG is formed when the last edge in the cycle is added; therefore the last process in the cycle to wait (the one from which that edge is outgoing) is certain to detect and resolve the cycle when it later runs CheckDeadLock. This holds even if that edge addition created multiple cycles; the process may indeed abort without ever noticing those additional cycles, but we don't particularly care. The only other possible creation of deadlocks is during deadlock resolution's rearrangement of wait queues, and we already saw that that algorithm will prove that it creates no new deadlocks before it attempts to actually execute any rearrangement. 2. It is not certain that a deadlock will be resolved by aborting the last-to-wait process. If earlier waiters in the cycle have not yet run CheckDeadLock, then the first one to do so will be the victim. 3. No live (wakable) process can be missed by ProcLockWakeup, since it examines every member of the wait queue (this was not true in the 7.0 implementation, BTW). Therefore, if ProcLockWakeup is always invoked after a lock is released or a wait queue is rearranged, there can be no failure to wake a wakable process. One should also note that LockErrorCleanup (abort a waiter due to outside factors) must run ProcLockWakeup, in case the canceled waiter was soft-blocking other waiters. 4. We can minimize excess rearrangement-trial work by being careful to scan the wait queue from the front when looking for soft edges. For example, if we have queue order A,B,C and C has deadlock conflicts with both A and B, we want to generate the "C before A" constraint first, rather than wasting time with "C before B", which won't move C far enough up. So we look for soft edges outgoing from C starting at the front of the wait queue. 5. The working data structures needed by the deadlock detection code can be limited to numbers of entries computed from MaxBackends. Therefore, we can allocate the worst-case space needed during backend startup. This seems a safer approach than trying to allocate workspace on the fly; we don't want to risk having the deadlock detector run out of memory, else we really have no guarantees at all that deadlock will be detected. 6. We abuse the deadlock detector to implement autovacuum cancellation. When we run the detector and we find that there's an autovacuum worker involved in the waits-for graph, we store a pointer to its PGPROC, and return a special return code (unless a hard deadlock has been detected). The caller can then send a cancellation signal. This implements the principle that autovacuum has a low locking priority (eg it must not block DDL on the table). Group Locking ------------- As if all of that weren't already complicated enough, PostgreSQL now supports parallelism (see src/backend/access/transam/README.parallel), which means that we might need to resolve deadlocks that occur between gangs of related processes rather than individual processes. This doesn't change the basic deadlock detection algorithm very much, but it makes the bookkeeping more complicated. We choose to regard locks held by processes in the same parallel group as non-conflicting. This means that two processes in a parallel group can hold a self-exclusive lock on the same relation at the same time, or one process can acquire an AccessShareLock while the other already holds AccessExclusiveLock. This might seem dangerous and could be in some cases (more on that below), but if we didn't do this then parallel query would be extremely prone to self-deadlock. For example, a parallel query against a relation on which the leader already had AccessExclusiveLock would hang, because the workers would try to lock the same relation and be blocked by the leader; yet the leader can't finish until it receives completion indications from all workers. An undetected deadlock results. This is far from the only scenario where such a problem happens. The same thing will occur if the leader holds only AccessShareLock, the worker seeks AccessShareLock, but between the time the leader attempts to acquire the lock and the time the worker attempts to acquire it, some other process queues up waiting for an AccessExclusiveLock. In this case, too, an indefinite hang results. It might seem that we could predict which locks the workers will attempt to acquire and ensure before going parallel that those locks would be acquired successfully. But this is very difficult to make work in a general way. For example, a parallel worker's portion of the query plan could involve an SQL-callable function which generates a query dynamically, and that query might happen to hit a table on which the leader happens to hold AccessExclusiveLock. By imposing enough restrictions on what workers can do, we could eventually create a situation where their behavior can be adequately restricted, but these restrictions would be fairly onerous, and even then, the system required to decide whether the workers will succeed at acquiring the necessary locks would be complex and possibly buggy. So, instead, we take the approach of deciding that locks within a lock group do not conflict. This eliminates the possibility of an undetected deadlock, but also opens up some problem cases: if the leader and worker try to do some operation at the same time which would ordinarily be prevented by the heavyweight lock mechanism, undefined behavior might result. In practice, the dangers are modest. The leader and worker share the same transaction, snapshot, and combo CID hash, and neither can perform any DDL or, indeed, write any data at all. Thus, for either to read a table locked exclusively by the other is safe enough. Problems would occur if the leader initiated parallelism from a point in the code at which it had some backend-private state that made table access from another process unsafe, for example after calling SetReindexProcessing and before calling ResetReindexProcessing, catastrophe could ensue, because the worker won't have that state. Similarly, problems could occur with certain kinds of non-relation locks, such as relation extension locks. It's no safer for two related processes to extend the same relation at the time than for unrelated processes to do the same. However, since parallel mode is strictly read-only at present, neither this nor most of the similar cases can arise at present. To allow parallel writes, we'll either need to (1) further enhance the deadlock detector to handle those types of locks in a different way than other types; or (2) have parallel workers use some other mutual exclusion method for such cases; or (3) revise those cases so that they no longer use heavyweight locking in the first place (which is not a crazy idea, given that such lock acquisitions are not expected to deadlock and that heavyweight lock acquisition is fairly slow anyway). Group locking adds three new members to each PGPROC: lockGroupLeader, lockGroupMembers, and lockGroupLink. A PGPROC's lockGroupLeader is NULL for processes not involved in parallel query. When a process wants to cooperate with parallel workers, it becomes a lock group leader, which means setting this field to point to its own PGPROC. When a parallel worker starts up, it points this field at the leader. The lockGroupMembers field is only used in the leader; it is a list of the member PGPROCs of the lock group (the leader and all workers). The lockGroupLink field is the list link for this list. All three of these fields are considered to be protected by a lock manager partition lock. The partition lock that protects these fields within a given lock group is chosen by taking the leader's pgprocno modulo the number of lock manager partitions. This unusual arrangement has a major advantage: the deadlock detector can count on the fact that no lockGroupLeader field can change while the deadlock detector is running, because it knows that it holds all the lock manager locks. Also, holding this single lock allows safe manipulation of the lockGroupMembers list for the lock group. We need an additional interlock when setting these fields, because a newly started parallel worker has to try to join the leader's lock group, but it has no guarantee that the group leader is still alive by the time it gets started. We try to ensure that the parallel leader dies after all workers in normal cases, but also that the system could survive relatively intact if that somehow fails to happen. This is one of the precautions against such a scenario: the leader relays its PGPROC and also its PID to the worker, and the worker fails to join the lock group unless the given PGPROC still has the same PID and is still a lock group leader. We assume that PIDs are not recycled quickly enough for this interlock to fail. User Locks (Advisory Locks) --------------------------- User locks are handled totally on the application side as long term cooperative locks which may extend beyond the normal transaction boundaries. Their purpose is to indicate to an application that someone is `working' on an item. So it is possible to put an user lock on a tuple's oid, retrieve the tuple, work on it for an hour and then update it and remove the lock. While the lock is active other clients can still read and write the tuple but they can be aware that it has been locked at the application level by someone. User locks and normal locks are completely orthogonal and they don't interfere with each other. User locks can be acquired either at session level or transaction level. A session-level lock request is not automatically released at transaction end, but must be explicitly released by the application. (However, any remaining locks are always released at session end.) Transaction-level user lock requests behave the same as normal lock requests, in that they are released at transaction end and do not need explicit unlocking. Locking during Hot Standby -------------------------- The Startup process is the only backend that can make changes during recovery, all other backends are read only. As a result the Startup process does not acquire locks on relations or objects except when the lock level is AccessExclusiveLock. Regular backends are only allowed to take locks on relations or objects at RowExclusiveLock or lower. This ensures that they do not conflict with each other or with the Startup process, unless AccessExclusiveLocks are requested by the Startup process. Deadlocks involving AccessExclusiveLocks are not possible, so we need not be concerned that a user initiated deadlock can prevent recovery from progressing. AccessExclusiveLocks on the primary or master node generate WAL records that are then applied by the Startup process. Locks are released at end of transaction just as they are in normal processing. These locks are held by the Startup process, acting as a proxy for the backends that originally acquired these locks. Again, these locks cannot conflict with one another, so the Startup process cannot deadlock itself either. Although deadlock is not possible, a regular backend's weak lock can prevent the Startup process from making progress in applying WAL, which is usually not something that should be tolerated for very long. Mechanisms exist to forcibly cancel a regular backend's query if it blocks the Startup process for too long.