]> granicus.if.org Git - postgresql/blob - src/backend/access/transam/varsup.c
Add basic infrastructure for 64 bit transaction IDs.
[postgresql] / src / backend / access / transam / varsup.c
1 /*-------------------------------------------------------------------------
2  *
3  * varsup.c
4  *        postgres OID & XID variables support routines
5  *
6  * Copyright (c) 2000-2019, PostgreSQL Global Development Group
7  *
8  * IDENTIFICATION
9  *        src/backend/access/transam/varsup.c
10  *
11  *-------------------------------------------------------------------------
12  */
13
14 #include "postgres.h"
15
16 #include "access/clog.h"
17 #include "access/commit_ts.h"
18 #include "access/subtrans.h"
19 #include "access/transam.h"
20 #include "access/xact.h"
21 #include "access/xlog.h"
22 #include "commands/dbcommands.h"
23 #include "miscadmin.h"
24 #include "postmaster/autovacuum.h"
25 #include "storage/pmsignal.h"
26 #include "storage/proc.h"
27 #include "utils/syscache.h"
28
29
30 /* Number of OIDs to prefetch (preallocate) per XLOG write */
31 #define VAR_OID_PREFETCH                8192
32
33 /* pointer to "variable cache" in shared memory (set up by shmem.c) */
34 VariableCache ShmemVariableCache = NULL;
35
36
37 /*
38  * Allocate the next XID for a new transaction or subtransaction.
39  *
40  * The new XID is also stored into MyPgXact before returning.
41  *
42  * Note: when this is called, we are actually already inside a valid
43  * transaction, since XIDs are now not allocated until the transaction
44  * does something.  So it is safe to do a database lookup if we want to
45  * issue a warning about XID wrap.
46  */
47 TransactionId
48 GetNewTransactionId(bool isSubXact)
49 {
50         TransactionId xid;
51
52         /*
53          * Workers synchronize transaction state at the beginning of each parallel
54          * operation, so we can't account for new XIDs after that point.
55          */
56         if (IsInParallelMode())
57                 elog(ERROR, "cannot assign TransactionIds during a parallel operation");
58
59         /*
60          * During bootstrap initialization, we return the special bootstrap
61          * transaction id.
62          */
63         if (IsBootstrapProcessingMode())
64         {
65                 Assert(!isSubXact);
66                 MyPgXact->xid = BootstrapTransactionId;
67                 return BootstrapTransactionId;
68         }
69
70         /* safety check, we should never get this far in a HS standby */
71         if (RecoveryInProgress())
72                 elog(ERROR, "cannot assign TransactionIds during recovery");
73
74         LWLockAcquire(XidGenLock, LW_EXCLUSIVE);
75
76         xid = XidFromFullTransactionId(ShmemVariableCache->nextFullXid);
77
78         /*----------
79          * Check to see if it's safe to assign another XID.  This protects against
80          * catastrophic data loss due to XID wraparound.  The basic rules are:
81          *
82          * If we're past xidVacLimit, start trying to force autovacuum cycles.
83          * If we're past xidWarnLimit, start issuing warnings.
84          * If we're past xidStopLimit, refuse to execute transactions, unless
85          * we are running in single-user mode (which gives an escape hatch
86          * to the DBA who somehow got past the earlier defenses).
87          *
88          * Note that this coding also appears in GetNewMultiXactId.
89          *----------
90          */
91         if (TransactionIdFollowsOrEquals(xid, ShmemVariableCache->xidVacLimit))
92         {
93                 /*
94                  * For safety's sake, we release XidGenLock while sending signals,
95                  * warnings, etc.  This is not so much because we care about
96                  * preserving concurrency in this situation, as to avoid any
97                  * possibility of deadlock while doing get_database_name(). First,
98                  * copy all the shared values we'll need in this path.
99                  */
100                 TransactionId xidWarnLimit = ShmemVariableCache->xidWarnLimit;
101                 TransactionId xidStopLimit = ShmemVariableCache->xidStopLimit;
102                 TransactionId xidWrapLimit = ShmemVariableCache->xidWrapLimit;
103                 Oid                     oldest_datoid = ShmemVariableCache->oldestXidDB;
104
105                 LWLockRelease(XidGenLock);
106
107                 /*
108                  * To avoid swamping the postmaster with signals, we issue the autovac
109                  * request only once per 64K transaction starts.  This still gives
110                  * plenty of chances before we get into real trouble.
111                  */
112                 if (IsUnderPostmaster && (xid % 65536) == 0)
113                         SendPostmasterSignal(PMSIGNAL_START_AUTOVAC_LAUNCHER);
114
115                 if (IsUnderPostmaster &&
116                         TransactionIdFollowsOrEquals(xid, xidStopLimit))
117                 {
118                         char       *oldest_datname = get_database_name(oldest_datoid);
119
120                         /* complain even if that DB has disappeared */
121                         if (oldest_datname)
122                                 ereport(ERROR,
123                                                 (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
124                                                  errmsg("database is not accepting commands to avoid wraparound data loss in database \"%s\"",
125                                                                 oldest_datname),
126                                                  errhint("Stop the postmaster and vacuum that database in single-user mode.\n"
127                                                                  "You might also need to commit or roll back old prepared transactions, or drop stale replication slots.")));
128                         else
129                                 ereport(ERROR,
130                                                 (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
131                                                  errmsg("database is not accepting commands to avoid wraparound data loss in database with OID %u",
132                                                                 oldest_datoid),
133                                                  errhint("Stop the postmaster and vacuum that database in single-user mode.\n"
134                                                                  "You might also need to commit or roll back old prepared transactions, or drop stale replication slots.")));
135                 }
136                 else if (TransactionIdFollowsOrEquals(xid, xidWarnLimit))
137                 {
138                         char       *oldest_datname = get_database_name(oldest_datoid);
139
140                         /* complain even if that DB has disappeared */
141                         if (oldest_datname)
142                                 ereport(WARNING,
143                                                 (errmsg("database \"%s\" must be vacuumed within %u transactions",
144                                                                 oldest_datname,
145                                                                 xidWrapLimit - xid),
146                                                  errhint("To avoid a database shutdown, execute a database-wide VACUUM in that database.\n"
147                                                                  "You might also need to commit or roll back old prepared transactions, or drop stale replication slots.")));
148                         else
149                                 ereport(WARNING,
150                                                 (errmsg("database with OID %u must be vacuumed within %u transactions",
151                                                                 oldest_datoid,
152                                                                 xidWrapLimit - xid),
153                                                  errhint("To avoid a database shutdown, execute a database-wide VACUUM in that database.\n"
154                                                                  "You might also need to commit or roll back old prepared transactions, or drop stale replication slots.")));
155                 }
156
157                 /* Re-acquire lock and start over */
158                 LWLockAcquire(XidGenLock, LW_EXCLUSIVE);
159                 xid = XidFromFullTransactionId(ShmemVariableCache->nextFullXid);
160         }
161
162         /*
163          * If we are allocating the first XID of a new page of the commit log,
164          * zero out that commit-log page before returning. We must do this while
165          * holding XidGenLock, else another xact could acquire and commit a later
166          * XID before we zero the page.  Fortunately, a page of the commit log
167          * holds 32K or more transactions, so we don't have to do this very often.
168          *
169          * Extend pg_subtrans and pg_commit_ts too.
170          */
171         ExtendCLOG(xid);
172         ExtendCommitTs(xid);
173         ExtendSUBTRANS(xid);
174
175         /*
176          * Now advance the nextFullXid counter.  This must not happen until after
177          * we have successfully completed ExtendCLOG() --- if that routine fails,
178          * we want the next incoming transaction to try it again.  We cannot
179          * assign more XIDs until there is CLOG space for them.
180          */
181         FullTransactionIdAdvance(&ShmemVariableCache->nextFullXid);
182
183         /*
184          * We must store the new XID into the shared ProcArray before releasing
185          * XidGenLock.  This ensures that every active XID older than
186          * latestCompletedXid is present in the ProcArray, which is essential for
187          * correct OldestXmin tracking; see src/backend/access/transam/README.
188          *
189          * Note that readers of PGXACT xid fields should be careful to fetch the
190          * value only once, rather than assume they can read a value multiple
191          * times and get the same answer each time.  Note we are assuming that
192          * TransactionId and int fetch/store are atomic.
193          *
194          * The same comments apply to the subxact xid count and overflow fields.
195          *
196          * Use of a write barrier prevents dangerous code rearrangement in this
197          * function; other backends could otherwise e.g. be examining my subxids
198          * info concurrently, and we don't want them to see an invalid
199          * intermediate state, such as an incremented nxids before the array entry
200          * is filled.
201          *
202          * Other processes that read nxids should do so before reading xids
203          * elements with a pg_read_barrier() in between, so that they can be sure
204          * not to read an uninitialized array element; see
205          * src/backend/storage/lmgr/README.barrier.
206          *
207          * If there's no room to fit a subtransaction XID into PGPROC, set the
208          * cache-overflowed flag instead.  This forces readers to look in
209          * pg_subtrans to map subtransaction XIDs up to top-level XIDs. There is a
210          * race-condition window, in that the new XID will not appear as running
211          * until its parent link has been placed into pg_subtrans. However, that
212          * will happen before anyone could possibly have a reason to inquire about
213          * the status of the XID, so it seems OK.  (Snapshots taken during this
214          * window *will* include the parent XID, so they will deliver the correct
215          * answer later on when someone does have a reason to inquire.)
216          */
217         if (!isSubXact)
218                 MyPgXact->xid = xid;    /* LWLockRelease acts as barrier */
219         else
220         {
221                 int                     nxids = MyPgXact->nxids;
222
223                 if (nxids < PGPROC_MAX_CACHED_SUBXIDS)
224                 {
225                         MyProc->subxids.xids[nxids] = xid;
226                         pg_write_barrier();
227                         MyPgXact->nxids = nxids + 1;
228                 }
229                 else
230                         MyPgXact->overflowed = true;
231         }
232
233         LWLockRelease(XidGenLock);
234
235         return xid;
236 }
237
238 /*
239  * Read nextFullXid but don't allocate it.
240  */
241 FullTransactionId
242 ReadNextFullTransactionId(void)
243 {
244         FullTransactionId fullXid;
245
246         LWLockAcquire(XidGenLock, LW_SHARED);
247         fullXid = ShmemVariableCache->nextFullXid;
248         LWLockRelease(XidGenLock);
249
250         return fullXid;
251 }
252
253 /*
254  * Advance nextFullXid to the value after a given xid.  The epoch is inferred.
255  * This must only be called during recovery or from two-phase start-up code.
256  */
257 void
258 AdvanceNextFullTransactionIdPastXid(TransactionId xid)
259 {
260         FullTransactionId newNextFullXid;
261         TransactionId next_xid;
262         uint32          epoch;
263
264         /*
265          * It is safe to read nextFullXid without a lock, because this is only
266          * called from the startup process or single-process mode, meaning that no
267          * other process can modify it.
268          */
269         Assert(AmStartupProcess() || !IsUnderPostmaster);
270
271         /* Fast return if this isn't an xid high enough to move the needle. */
272         next_xid = XidFromFullTransactionId(ShmemVariableCache->nextFullXid);
273         if (!TransactionIdFollowsOrEquals(xid, next_xid))
274                 return;
275
276         /*
277          * Compute the FullTransactionId that comes after the given xid.  To do
278          * this, we preserve the existing epoch, but detect when we've wrapped
279          * into a new epoch.  This is necessary because WAL records and 2PC state
280          * currently contain 32 bit xids.  The wrap logic is safe in those cases
281          * because the span of active xids cannot exceed one epoch at any given
282          * point in the WAL stream.
283          */
284         TransactionIdAdvance(xid);
285         epoch = EpochFromFullTransactionId(ShmemVariableCache->nextFullXid);
286         if (unlikely(xid < next_xid))
287                 ++epoch;
288         newNextFullXid = FullTransactionIdFromEpochAndXid(epoch, xid);
289
290         /*
291          * We still need to take a lock to modify the value when there are
292          * concurrent readers.
293          */
294         LWLockAcquire(XidGenLock, LW_EXCLUSIVE);
295         ShmemVariableCache->nextFullXid = newNextFullXid;
296         LWLockRelease(XidGenLock);
297 }
298
299 /*
300  * Advance the cluster-wide value for the oldest valid clog entry.
301  *
302  * We must acquire CLogTruncationLock to advance the oldestClogXid. It's not
303  * necessary to hold the lock during the actual clog truncation, only when we
304  * advance the limit, as code looking up arbitrary xids is required to hold
305  * CLogTruncationLock from when it tests oldestClogXid through to when it
306  * completes the clog lookup.
307  */
308 void
309 AdvanceOldestClogXid(TransactionId oldest_datfrozenxid)
310 {
311         LWLockAcquire(CLogTruncationLock, LW_EXCLUSIVE);
312         if (TransactionIdPrecedes(ShmemVariableCache->oldestClogXid,
313                                                           oldest_datfrozenxid))
314         {
315                 ShmemVariableCache->oldestClogXid = oldest_datfrozenxid;
316         }
317         LWLockRelease(CLogTruncationLock);
318 }
319
320 /*
321  * Determine the last safe XID to allocate using the currently oldest
322  * datfrozenxid (ie, the oldest XID that might exist in any database
323  * of our cluster), and the OID of the (or a) database with that value.
324  */
325 void
326 SetTransactionIdLimit(TransactionId oldest_datfrozenxid, Oid oldest_datoid)
327 {
328         TransactionId xidVacLimit;
329         TransactionId xidWarnLimit;
330         TransactionId xidStopLimit;
331         TransactionId xidWrapLimit;
332         TransactionId curXid;
333
334         Assert(TransactionIdIsNormal(oldest_datfrozenxid));
335
336         /*
337          * The place where we actually get into deep trouble is halfway around
338          * from the oldest potentially-existing XID.  (This calculation is
339          * probably off by one or two counts, because the special XIDs reduce the
340          * size of the loop a little bit.  But we throw in plenty of slop below,
341          * so it doesn't matter.)
342          */
343         xidWrapLimit = oldest_datfrozenxid + (MaxTransactionId >> 1);
344         if (xidWrapLimit < FirstNormalTransactionId)
345                 xidWrapLimit += FirstNormalTransactionId;
346
347         /*
348          * We'll refuse to continue assigning XIDs in interactive mode once we get
349          * within 1M transactions of data loss.  This leaves lots of room for the
350          * DBA to fool around fixing things in a standalone backend, while not
351          * being significant compared to total XID space. (Note that since
352          * vacuuming requires one transaction per table cleaned, we had better be
353          * sure there's lots of XIDs left...)
354          */
355         xidStopLimit = xidWrapLimit - 1000000;
356         if (xidStopLimit < FirstNormalTransactionId)
357                 xidStopLimit -= FirstNormalTransactionId;
358
359         /*
360          * We'll start complaining loudly when we get within 10M transactions of
361          * the stop point.  This is kind of arbitrary, but if you let your gas
362          * gauge get down to 1% of full, would you be looking for the next gas
363          * station?  We need to be fairly liberal about this number because there
364          * are lots of scenarios where most transactions are done by automatic
365          * clients that won't pay attention to warnings. (No, we're not gonna make
366          * this configurable.  If you know enough to configure it, you know enough
367          * to not get in this kind of trouble in the first place.)
368          */
369         xidWarnLimit = xidStopLimit - 10000000;
370         if (xidWarnLimit < FirstNormalTransactionId)
371                 xidWarnLimit -= FirstNormalTransactionId;
372
373         /*
374          * We'll start trying to force autovacuums when oldest_datfrozenxid gets
375          * to be more than autovacuum_freeze_max_age transactions old.
376          *
377          * Note: guc.c ensures that autovacuum_freeze_max_age is in a sane range,
378          * so that xidVacLimit will be well before xidWarnLimit.
379          *
380          * Note: autovacuum_freeze_max_age is a PGC_POSTMASTER parameter so that
381          * we don't have to worry about dealing with on-the-fly changes in its
382          * value.  It doesn't look practical to update shared state from a GUC
383          * assign hook (too many processes would try to execute the hook,
384          * resulting in race conditions as well as crashes of those not connected
385          * to shared memory).  Perhaps this can be improved someday.  See also
386          * SetMultiXactIdLimit.
387          */
388         xidVacLimit = oldest_datfrozenxid + autovacuum_freeze_max_age;
389         if (xidVacLimit < FirstNormalTransactionId)
390                 xidVacLimit += FirstNormalTransactionId;
391
392         /* Grab lock for just long enough to set the new limit values */
393         LWLockAcquire(XidGenLock, LW_EXCLUSIVE);
394         ShmemVariableCache->oldestXid = oldest_datfrozenxid;
395         ShmemVariableCache->xidVacLimit = xidVacLimit;
396         ShmemVariableCache->xidWarnLimit = xidWarnLimit;
397         ShmemVariableCache->xidStopLimit = xidStopLimit;
398         ShmemVariableCache->xidWrapLimit = xidWrapLimit;
399         ShmemVariableCache->oldestXidDB = oldest_datoid;
400         curXid = XidFromFullTransactionId(ShmemVariableCache->nextFullXid);
401         LWLockRelease(XidGenLock);
402
403         /* Log the info */
404         ereport(DEBUG1,
405                         (errmsg("transaction ID wrap limit is %u, limited by database with OID %u",
406                                         xidWrapLimit, oldest_datoid)));
407
408         /*
409          * If past the autovacuum force point, immediately signal an autovac
410          * request.  The reason for this is that autovac only processes one
411          * database per invocation.  Once it's finished cleaning up the oldest
412          * database, it'll call here, and we'll signal the postmaster to start
413          * another iteration immediately if there are still any old databases.
414          */
415         if (TransactionIdFollowsOrEquals(curXid, xidVacLimit) &&
416                 IsUnderPostmaster && !InRecovery)
417                 SendPostmasterSignal(PMSIGNAL_START_AUTOVAC_LAUNCHER);
418
419         /* Give an immediate warning if past the wrap warn point */
420         if (TransactionIdFollowsOrEquals(curXid, xidWarnLimit) && !InRecovery)
421         {
422                 char       *oldest_datname;
423
424                 /*
425                  * We can be called when not inside a transaction, for example during
426                  * StartupXLOG().  In such a case we cannot do database access, so we
427                  * must just report the oldest DB's OID.
428                  *
429                  * Note: it's also possible that get_database_name fails and returns
430                  * NULL, for example because the database just got dropped.  We'll
431                  * still warn, even though the warning might now be unnecessary.
432                  */
433                 if (IsTransactionState())
434                         oldest_datname = get_database_name(oldest_datoid);
435                 else
436                         oldest_datname = NULL;
437
438                 if (oldest_datname)
439                         ereport(WARNING,
440                                         (errmsg("database \"%s\" must be vacuumed within %u transactions",
441                                                         oldest_datname,
442                                                         xidWrapLimit - curXid),
443                                          errhint("To avoid a database shutdown, execute a database-wide VACUUM in that database.\n"
444                                                          "You might also need to commit or roll back old prepared transactions, or drop stale replication slots.")));
445                 else
446                         ereport(WARNING,
447                                         (errmsg("database with OID %u must be vacuumed within %u transactions",
448                                                         oldest_datoid,
449                                                         xidWrapLimit - curXid),
450                                          errhint("To avoid a database shutdown, execute a database-wide VACUUM in that database.\n"
451                                                          "You might also need to commit or roll back old prepared transactions, or drop stale replication slots.")));
452         }
453 }
454
455
456 /*
457  * ForceTransactionIdLimitUpdate -- does the XID wrap-limit data need updating?
458  *
459  * We primarily check whether oldestXidDB is valid.  The cases we have in
460  * mind are that that database was dropped, or the field was reset to zero
461  * by pg_resetwal.  In either case we should force recalculation of the
462  * wrap limit.  Also do it if oldestXid is old enough to be forcing
463  * autovacuums or other actions; this ensures we update our state as soon
464  * as possible once extra overhead is being incurred.
465  */
466 bool
467 ForceTransactionIdLimitUpdate(void)
468 {
469         TransactionId nextXid;
470         TransactionId xidVacLimit;
471         TransactionId oldestXid;
472         Oid                     oldestXidDB;
473
474         /* Locking is probably not really necessary, but let's be careful */
475         LWLockAcquire(XidGenLock, LW_SHARED);
476         nextXid = XidFromFullTransactionId(ShmemVariableCache->nextFullXid);
477         xidVacLimit = ShmemVariableCache->xidVacLimit;
478         oldestXid = ShmemVariableCache->oldestXid;
479         oldestXidDB = ShmemVariableCache->oldestXidDB;
480         LWLockRelease(XidGenLock);
481
482         if (!TransactionIdIsNormal(oldestXid))
483                 return true;                    /* shouldn't happen, but just in case */
484         if (!TransactionIdIsValid(xidVacLimit))
485                 return true;                    /* this shouldn't happen anymore either */
486         if (TransactionIdFollowsOrEquals(nextXid, xidVacLimit))
487                 return true;                    /* past VacLimit, don't delay updating */
488         if (!SearchSysCacheExists1(DATABASEOID, ObjectIdGetDatum(oldestXidDB)))
489                 return true;                    /* could happen, per comments above */
490         return false;
491 }
492
493
494 /*
495  * GetNewObjectId -- allocate a new OID
496  *
497  * OIDs are generated by a cluster-wide counter.  Since they are only 32 bits
498  * wide, counter wraparound will occur eventually, and therefore it is unwise
499  * to assume they are unique unless precautions are taken to make them so.
500  * Hence, this routine should generally not be used directly.  The only direct
501  * callers should be GetNewOidWithIndex() and GetNewRelFileNode() in
502  * catalog/catalog.c.
503  */
504 Oid
505 GetNewObjectId(void)
506 {
507         Oid                     result;
508
509         /* safety check, we should never get this far in a HS standby */
510         if (RecoveryInProgress())
511                 elog(ERROR, "cannot assign OIDs during recovery");
512
513         LWLockAcquire(OidGenLock, LW_EXCLUSIVE);
514
515         /*
516          * Check for wraparound of the OID counter.  We *must* not return 0
517          * (InvalidOid), and in normal operation we mustn't return anything below
518          * FirstNormalObjectId since that range is reserved for initdb (see
519          * IsCatalogClass()).  Note we are relying on unsigned comparison.
520          *
521          * During initdb, we start the OID generator at FirstBootstrapObjectId, so
522          * we only wrap if before that point when in bootstrap or standalone mode.
523          * The first time through this routine after normal postmaster start, the
524          * counter will be forced up to FirstNormalObjectId.  This mechanism
525          * leaves the OIDs between FirstBootstrapObjectId and FirstNormalObjectId
526          * available for automatic assignment during initdb, while ensuring they
527          * will never conflict with user-assigned OIDs.
528          */
529         if (ShmemVariableCache->nextOid < ((Oid) FirstNormalObjectId))
530         {
531                 if (IsPostmasterEnvironment)
532                 {
533                         /* wraparound, or first post-initdb assignment, in normal mode */
534                         ShmemVariableCache->nextOid = FirstNormalObjectId;
535                         ShmemVariableCache->oidCount = 0;
536                 }
537                 else
538                 {
539                         /* we may be bootstrapping, so don't enforce the full range */
540                         if (ShmemVariableCache->nextOid < ((Oid) FirstBootstrapObjectId))
541                         {
542                                 /* wraparound in standalone mode (unlikely but possible) */
543                                 ShmemVariableCache->nextOid = FirstNormalObjectId;
544                                 ShmemVariableCache->oidCount = 0;
545                         }
546                 }
547         }
548
549         /* If we run out of logged for use oids then we must log more */
550         if (ShmemVariableCache->oidCount == 0)
551         {
552                 XLogPutNextOid(ShmemVariableCache->nextOid + VAR_OID_PREFETCH);
553                 ShmemVariableCache->oidCount = VAR_OID_PREFETCH;
554         }
555
556         result = ShmemVariableCache->nextOid;
557
558         (ShmemVariableCache->nextOid)++;
559         (ShmemVariableCache->oidCount)--;
560
561         LWLockRelease(OidGenLock);
562
563         return result;
564 }