-<!-- $Header: /cvsroot/pgsql/doc/src/sgml/backup.sgml,v 2.11 2001/08/16 04:30:41 ishii Exp $ -->
+<!-- $Header: /cvsroot/pgsql/doc/src/sgml/backup.sgml,v 2.12 2001/08/25 18:52:41 tgl Exp $ -->
<chapter id="backup">
<title>Backup and Restore</title>
individual tables or databases from their respective files or
directories. This will <emphasis>not</> work because the
information contained in these files contains only half the
- truth. The other half is in the file
- <filename>pg_log</filename>, which contains the commit status of
+ truth. The other half is in the commit log files
+ <filename>pg_clog/*</filename>, which contain the commit status of
all transactions. A table file is only usable with this
information. Of course it is also impossible to restore only a
- table and the associated <filename>pg_log</filename> file
+ table and the associated <filename>pg_clog</filename> data
because that will render all other tables in the database
cluster useless.
</para>
-<!-- $Header: /cvsroot/pgsql/doc/src/sgml/wal.sgml,v 1.7 2001/05/17 21:50:16 petere Exp $ -->
+<!-- $Header: /cvsroot/pgsql/doc/src/sgml/wal.sgml,v 1.8 2001/08/25 18:52:41 tgl Exp $ -->
<chapter id="wal">
<title>Write-Ahead Logging (<acronym>WAL</acronym>)</title>
In this first release of <acronym>WAL</acronym>, UNDO operation is
not implemented, because of lack of time. This means that changes
made by aborted transactions will still occupy disk space and that
- we still need a permanent <filename>pg_log</filename> file to hold
+ we still need a permanent <filename>pg_clog</filename> file to hold
the status of transactions, since we are not able to re-use
transaction identifiers. Once UNDO is implemented,
- <filename>pg_log</filename> will no longer be required to be
+ <filename>pg_clog</filename> will no longer be required to be
permanent; it will be possible to remove
- <filename>pg_log</filename> at shutdown, split it into segments
+ <filename>pg_clog</filename> at shutdown, split it into segments
and remove old segments.
</para>
# Makefile for access/transam
#
# IDENTIFICATION
-# $Header: /cvsroot/pgsql/src/backend/access/transam/Makefile,v 1.14 2001/08/24 14:07:48 petere Exp $
+# $Header: /cvsroot/pgsql/src/backend/access/transam/Makefile,v 1.15 2001/08/25 18:52:41 tgl Exp $
#
#-------------------------------------------------------------------------
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = transam.o transsup.o varsup.o xact.o xid.o xlog.o xlogutils.o rmgr.o
+OBJS = clog.o transam.o varsup.o xact.o xid.o xlog.o xlogutils.o rmgr.o
all: SUBSYS.o
--- /dev/null
+/*-------------------------------------------------------------------------
+ *
+ * clog.c
+ * PostgreSQL transaction-commit-log manager
+ *
+ * This module replaces the old "pg_log" access code, which treated pg_log
+ * essentially like a relation, in that it went through the regular buffer
+ * manager. The problem with that was that there wasn't any good way to
+ * recycle storage space for transactions so old that they'll never be
+ * looked up again. Now we use specialized access code so that the commit
+ * log can be broken into relatively small, independent segments.
+ *
+ * Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * $Header: /cvsroot/pgsql/src/backend/access/transam/clog.c,v 1.1 2001/08/25 18:52:41 tgl Exp $
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include <fcntl.h>
+#include <dirent.h>
+#include <errno.h>
+#include <unistd.h>
+
+#include "access/clog.h"
+#include "storage/s_lock.h"
+#include "miscadmin.h"
+
+
+/*
+ * Defines for CLOG page and segment sizes. A page is the same BLCKSZ
+ * as is used everywhere else in Postgres. The CLOG segment size can be
+ * chosen somewhat arbitrarily; we make it 1 million transactions by default,
+ * or 256Kb.
+ *
+ * Note: because TransactionIds are 32 bits and wrap around at 0xFFFFFFFF,
+ * CLOG page numbering also wraps around at 0xFFFFFFFF/CLOG_XACTS_PER_PAGE,
+ * and CLOG segment numbering at 0xFFFFFFFF/CLOG_XACTS_PER_SEGMENT. We need
+ * take no explicit notice of that fact in this module, except when comparing
+ * segment and page numbers in TruncateCLOG (see CLOGPagePrecedes).
+ */
+
+#define CLOG_BLCKSZ BLCKSZ
+
+/* We need two bits per xact, so four xacts fit in a byte */
+#define CLOG_BITS_PER_XACT 2
+#define CLOG_XACTS_PER_BYTE 4
+#define CLOG_XACTS_PER_PAGE (CLOG_BLCKSZ * CLOG_XACTS_PER_BYTE)
+#define CLOG_XACT_BITMASK ((1 << CLOG_BITS_PER_XACT) - 1)
+
+#define CLOG_XACTS_PER_SEGMENT 0x100000
+#define CLOG_PAGES_PER_SEGMENT (CLOG_XACTS_PER_SEGMENT / CLOG_XACTS_PER_PAGE)
+
+#define TransactionIdToPage(xid) ((xid) / (TransactionId) CLOG_XACTS_PER_PAGE)
+#define TransactionIdToPgIndex(xid) ((xid) % (TransactionId) CLOG_XACTS_PER_PAGE)
+#define TransactionIdToByte(xid) (TransactionIdToPgIndex(xid) / CLOG_XACTS_PER_BYTE)
+#define TransactionIdToBIndex(xid) ((xid) % (TransactionId) CLOG_XACTS_PER_BYTE)
+
+
+/*----------
+ * Shared-memory data structures for CLOG control
+ *
+ * We use a simple least-recently-used scheme to manage a pool of page
+ * buffers for the CLOG. Under ordinary circumstances we expect that write
+ * traffic will occur mostly to the latest CLOG page (and to the just-prior
+ * page, soon after a page transition). Read traffic will probably touch
+ * a larger span of pages, but in any case a fairly small number of page
+ * buffers should be sufficient. So, we just search the buffers using plain
+ * linear search; there's no need for a hashtable or anything fancy.
+ * The management algorithm is straight LRU except that we will never swap
+ * out the latest page (since we know it's going to be hit again eventually).
+ *
+ * We use an overall spinlock to protect the shared data structures, plus
+ * per-buffer spinlocks that synchronize I/O for each buffer. A process
+ * that is reading in or writing out a page buffer does not hold the control
+ * lock, only the per-buffer lock for the buffer it is working on.
+ *
+ * To change the page number or state of a buffer, one must normally hold
+ * the control lock. (The sole exception to this rule is that a writer
+ * process changes the state from DIRTY to WRITE_IN_PROGRESS while holding
+ * only the per-buffer lock.) If the buffer's state is neither EMPTY nor
+ * CLEAN, then there may be processes doing (or waiting to do) I/O on the
+ * buffer, so the page number may not be changed, and the only allowed state
+ * transition is to change WRITE_IN_PROGRESS to DIRTY after dirtying the page.
+ * To do any other state transition involving a buffer with potential I/O
+ * processes, one must hold both the per-buffer lock and the control lock.
+ * (Note the control lock must be acquired second; do not wait on a buffer
+ * lock while holding the control lock.) A process wishing to read a page
+ * marks the buffer state as READ_IN_PROGRESS, then drops the control lock,
+ * acquires the per-buffer lock, and rechecks the state before proceeding.
+ * This recheck takes care of the possibility that someone else already did
+ * the read, while the early marking prevents someone else from trying to
+ * read the same page into a different buffer.
+ *
+ * Note we are assuming that read and write of the state value is atomic,
+ * since I/O processes may examine and change the state while not holding
+ * the control lock.
+ *
+ * As with the regular buffer manager, it is possible for another process
+ * to re-dirty a page that is currently being written out. This is handled
+ * by setting the page's state from WRITE_IN_PROGRESS to DIRTY. The writing
+ * process must notice this and not mark the page CLEAN when it's done.
+ *
+ * XXX it's probably okay to use a spinlock for the control lock, since
+ * that lock is only held for very short operations. It'd be nice to use
+ * some other form of lock for the per-buffer I/O locks, however.
+ *
+ * XLOG interactions: this module generates an XLOG record whenever a new
+ * CLOG page is initialized to zeroes. Other writes of CLOG come from
+ * recording of transaction commit or abort in xact.c, which generates its
+ * own XLOG records for these events and will re-perform the status update
+ * on redo; so we need make no additional XLOG entry here. Also, the XLOG
+ * is guaranteed flushed through the XLOG commit record before we are called
+ * to log a commit, so the WAL rule "write xlog before data" is satisfied
+ * automatically for commits, and we don't really care for aborts. Therefore,
+ * we don't need to mark XLOG pages with LSN information; we have enough
+ * synchronization already.
+ *----------
+ */
+#define NUM_CLOG_BUFFERS 8
+
+typedef enum
+{
+ CLOG_PAGE_EMPTY, /* CLOG buffer is not in use */
+ CLOG_PAGE_READ_IN_PROGRESS, /* CLOG page is being read in */
+ CLOG_PAGE_CLEAN, /* CLOG page is valid and not dirty */
+ CLOG_PAGE_DIRTY, /* CLOG page is valid but needs write */
+ CLOG_PAGE_WRITE_IN_PROGRESS /* CLOG page is being written out in */
+} ClogPageStatus;
+
+/*
+ * Shared-memory state for CLOG.
+ */
+typedef struct ClogCtlData
+{
+ /*
+ * Info for each buffer slot. Page number is undefined when status is
+ * EMPTY. lru_count is essentially the number of operations since last
+ * use of this page; the page with highest lru_count is the best candidate
+ * to replace.
+ */
+ char *page_buffer[NUM_CLOG_BUFFERS];
+ ClogPageStatus page_status[NUM_CLOG_BUFFERS];
+ int page_number[NUM_CLOG_BUFFERS];
+ unsigned int page_lru_count[NUM_CLOG_BUFFERS];
+ /*
+ * latest_page_number is the page number of the current end of the
+ * CLOG; this is not critical data, since we use it only to avoid
+ * swapping out the latest page.
+ */
+ int latest_page_number;
+
+ slock_t control_lck; /* Lock for ClogCtlData itself */
+ slock_t buffer_lck[NUM_CLOG_BUFFERS]; /* Per-buffer I/O locks */
+} ClogCtlData;
+
+static ClogCtlData *ClogCtl = NULL;
+
+/*
+ * ClogDir is set during CLOGShmemInit and does not change thereafter.
+ * The value is automatically inherited by backends via fork, and
+ * doesn't need to be in shared memory.
+ */
+static char ClogDir[MAXPGPATH];
+
+#define ClogFileName(path, seg) \
+ snprintf(path, MAXPGPATH, "%s/%04X", ClogDir, seg)
+
+/*
+ * Macro to mark a buffer slot "most recently used".
+ */
+#define ClogRecentlyUsed(slotno) \
+ do { \
+ int iilru; \
+ for (iilru = 0; iilru < NUM_CLOG_BUFFERS; iilru++) \
+ ClogCtl->page_lru_count[iilru]++; \
+ ClogCtl->page_lru_count[slotno] = 0; \
+ } while (0)
+
+
+static int ZeroCLOGPage(int pageno, bool writeXlog);
+static int ReadCLOGPage(int pageno);
+static void WriteCLOGPage(int slotno);
+static void CLOGPhysicalReadPage(int pageno, int slotno);
+static void CLOGPhysicalWritePage(int pageno, int slotno);
+static int SelectLRUCLOGPage(int pageno);
+static bool ScanCLOGDirectory(int cutoffPage, bool doDeletions);
+static bool CLOGPagePrecedes(int page1, int page2);
+static void WriteZeroPageXlogRec(int pageno);
+
+
+/*
+ * Record the final state of a transaction in the commit log.
+ *
+ * NB: this is a low-level routine and is NOT the preferred entry point
+ * for most uses; TransactionLogUpdate() in transam.c is the intended caller.
+ */
+void
+TransactionIdSetStatus(TransactionId xid, XidStatus status)
+{
+ int pageno = TransactionIdToPage(xid);
+ int byteno = TransactionIdToByte(xid);
+ int bshift = TransactionIdToBIndex(xid) * CLOG_BITS_PER_XACT;
+ int slotno;
+ char *byteptr;
+
+ Assert(status == TRANSACTION_STATUS_COMMITTED ||
+ status == TRANSACTION_STATUS_ABORTED);
+
+ S_LOCK(&(ClogCtl->control_lck));
+
+ slotno = ReadCLOGPage(pageno);
+ byteptr = ClogCtl->page_buffer[slotno] + byteno;
+
+ /* Current state should be 0 or target state */
+ Assert(((*byteptr >> bshift) & CLOG_XACT_BITMASK) == 0 ||
+ ((*byteptr >> bshift) & CLOG_XACT_BITMASK) == status);
+
+ *byteptr |= (status << bshift);
+
+ ClogCtl->page_status[slotno] = CLOG_PAGE_DIRTY;
+
+ S_UNLOCK(&(ClogCtl->control_lck));
+}
+
+/*
+ * Interrogate the state of a transaction in the commit log.
+ *
+ * NB: this is a low-level routine and is NOT the preferred entry point
+ * for most uses; TransactionLogTest() in transam.c is the intended caller.
+ */
+XidStatus
+TransactionIdGetStatus(TransactionId xid)
+{
+ int pageno = TransactionIdToPage(xid);
+ int byteno = TransactionIdToByte(xid);
+ int bshift = TransactionIdToBIndex(xid) * CLOG_BITS_PER_XACT;
+ int slotno;
+ char *byteptr;
+ XidStatus status;
+
+ S_LOCK(&(ClogCtl->control_lck));
+
+ slotno = ReadCLOGPage(pageno);
+ byteptr = ClogCtl->page_buffer[slotno] + byteno;
+
+ status = (*byteptr >> bshift) & CLOG_XACT_BITMASK;
+
+ S_UNLOCK(&(ClogCtl->control_lck));
+
+ return status;
+}
+
+
+/*
+ * Initialization of shared memory for CLOG
+ */
+
+int
+CLOGShmemSize(void)
+{
+ return MAXALIGN(sizeof(ClogCtlData) + CLOG_BLCKSZ * NUM_CLOG_BUFFERS);
+}
+
+void
+CLOGShmemInit(void)
+{
+ bool found;
+ char *bufptr;
+ int slotno;
+
+ /* this must agree with space requested by CLOGShmemSize() */
+ ClogCtl = (ClogCtlData *)
+ ShmemInitStruct("CLOG Ctl",
+ MAXALIGN(sizeof(ClogCtlData) +
+ CLOG_BLCKSZ * NUM_CLOG_BUFFERS),
+ &found);
+ Assert(!found);
+
+ memset(ClogCtl, 0, sizeof(ClogCtlData));
+
+ S_INIT_LOCK(&(ClogCtl->control_lck));
+
+ bufptr = ((char *) ClogCtl) + sizeof(ClogCtlData);
+
+ for (slotno = 0; slotno < NUM_CLOG_BUFFERS; slotno++)
+ {
+ ClogCtl->page_buffer[slotno] = bufptr;
+ ClogCtl->page_status[slotno] = CLOG_PAGE_EMPTY;
+ S_INIT_LOCK(&(ClogCtl->buffer_lck[slotno]));
+ bufptr += CLOG_BLCKSZ;
+ }
+
+ /* ClogCtl->latest_page_number will be set later */
+
+ /* Init CLOG directory path */
+ snprintf(ClogDir, MAXPGPATH, "%s/pg_clog", DataDir);
+}
+
+/*
+ * This func must be called ONCE on system install. It creates
+ * the initial CLOG segment. (The CLOG directory is assumed to
+ * have been created by the initdb shell script, and CLOGShmemInit
+ * must have been called already.)
+ */
+void
+BootStrapCLOG(void)
+{
+ int slotno;
+
+ S_LOCK(&(ClogCtl->control_lck));
+
+ /* Create and zero the first page of the commit log */
+ slotno = ZeroCLOGPage(0, false);
+
+ /* Make sure it's written out */
+ WriteCLOGPage(slotno);
+ Assert(ClogCtl->page_status[slotno] == CLOG_PAGE_CLEAN);
+
+ S_UNLOCK(&(ClogCtl->control_lck));
+}
+
+/*
+ * Initialize (or reinitialize) a page of CLOG to zeroes.
+ * If writeXlog is TRUE, also emit an XLOG record saying we did this.
+ *
+ * The page is not actually written, just set up in shared memory.
+ * The slot number of the new page is returned.
+ *
+ * Control lock must be held at entry, and will be held at exit.
+ */
+static int
+ZeroCLOGPage(int pageno, bool writeXlog)
+{
+ int slotno;
+
+ /* Find a suitable buffer slot for the page */
+ slotno = SelectLRUCLOGPage(pageno);
+ Assert(ClogCtl->page_status[slotno] == CLOG_PAGE_EMPTY ||
+ ClogCtl->page_status[slotno] == CLOG_PAGE_CLEAN ||
+ ClogCtl->page_number[slotno] == pageno);
+
+ /* Mark the slot as containing this page */
+ ClogCtl->page_number[slotno] = pageno;
+ ClogCtl->page_status[slotno] = CLOG_PAGE_DIRTY;
+ ClogRecentlyUsed(slotno);
+
+ /* Set the buffer to zeroes */
+ MemSet(ClogCtl->page_buffer[slotno], 0, CLOG_BLCKSZ);
+
+ /* Assume this page is now the latest active page */
+ ClogCtl->latest_page_number = pageno;
+
+ if (writeXlog)
+ WriteZeroPageXlogRec(pageno);
+
+ return slotno;
+}
+
+/*
+ * Find a CLOG page in a shared buffer, reading it in if necessary.
+ * The page number must correspond to an already-initialized page.
+ *
+ * Return value is the shared-buffer slot number now holding the page.
+ * The buffer's LRU access info is updated.
+ *
+ * Control lock must be held at entry, and will be held at exit.
+ */
+static int
+ReadCLOGPage(int pageno)
+{
+ /* Outer loop handles restart if we lose the buffer to someone else */
+ for (;;)
+ {
+ int slotno;
+
+ /* See if page already is in memory; if not, pick victim slot */
+ slotno = SelectLRUCLOGPage(pageno);
+
+ /* Did we find the page in memory? */
+ if (ClogCtl->page_number[slotno] == pageno &&
+ ClogCtl->page_status[slotno] != CLOG_PAGE_EMPTY)
+ {
+ /* If page is still being read in, we cannot use it yet */
+ if (ClogCtl->page_status[slotno] != CLOG_PAGE_READ_IN_PROGRESS)
+ {
+ /* otherwise, it's ready to use */
+ ClogRecentlyUsed(slotno);
+ return slotno;
+ }
+ }
+ else
+ {
+ /* We found no match; assert we selected a freeable slot */
+ Assert(ClogCtl->page_status[slotno] == CLOG_PAGE_EMPTY ||
+ ClogCtl->page_status[slotno] == CLOG_PAGE_CLEAN);
+ }
+
+ /* Mark the slot read-busy (no-op if it already was) */
+ ClogCtl->page_number[slotno] = pageno;
+ ClogCtl->page_status[slotno] = CLOG_PAGE_READ_IN_PROGRESS;
+
+ /*
+ * Temporarily mark page as recently-used to discourage
+ * SelectLRUCLOGPage from selecting it again for someone else.
+ */
+ ClogCtl->page_lru_count[slotno] = 0;
+
+ /* Release shared lock, grab per-buffer lock instead */
+ S_UNLOCK(&(ClogCtl->control_lck));
+ S_LOCK(&(ClogCtl->buffer_lck[slotno]));
+
+ /*
+ * Check to see if someone else already did the read, or took the
+ * buffer away from us. If so, restart from the top.
+ */
+ if (ClogCtl->page_number[slotno] != pageno ||
+ ClogCtl->page_status[slotno] != CLOG_PAGE_READ_IN_PROGRESS)
+ {
+ S_UNLOCK(&(ClogCtl->buffer_lck[slotno]));
+ S_LOCK(&(ClogCtl->control_lck));
+ continue;
+ }
+
+ /* Okay, do the read */
+ CLOGPhysicalReadPage(pageno, slotno);
+
+ /* Re-acquire shared control lock and update page state */
+ S_LOCK(&(ClogCtl->control_lck));
+
+ Assert(ClogCtl->page_number[slotno] == pageno &&
+ ClogCtl->page_status[slotno] == CLOG_PAGE_READ_IN_PROGRESS);
+
+ ClogCtl->page_status[slotno] = CLOG_PAGE_CLEAN;
+
+ S_UNLOCK(&(ClogCtl->buffer_lck[slotno]));
+
+ ClogRecentlyUsed(slotno);
+ return slotno;
+ }
+}
+
+/*
+ * Write a CLOG page from a shared buffer, if necessary.
+ * Does nothing if the specified slot is not dirty.
+ *
+ * NOTE: only one write attempt is made here. Hence, it is possible that
+ * the page is still dirty at exit (if someone else re-dirtied it during
+ * the write). However, we *do* attempt a fresh write even if the page
+ * is already being written; this is for checkpoints.
+ *
+ * Control lock must be held at entry, and will be held at exit.
+ */
+static void
+WriteCLOGPage(int slotno)
+{
+ int pageno;
+
+ /* Do nothing if page does not need writing */
+ if (ClogCtl->page_status[slotno] != CLOG_PAGE_DIRTY &&
+ ClogCtl->page_status[slotno] != CLOG_PAGE_WRITE_IN_PROGRESS)
+ return;
+
+ pageno = ClogCtl->page_number[slotno];
+
+ /* Release shared lock, grab per-buffer lock instead */
+ S_UNLOCK(&(ClogCtl->control_lck));
+ S_LOCK(&(ClogCtl->buffer_lck[slotno]));
+
+ /*
+ * Check to see if someone else already did the write, or took the
+ * buffer away from us. If so, do nothing. NOTE: we really should
+ * never see WRITE_IN_PROGRESS here, since that state should only
+ * occur while the writer is holding the buffer lock. But accept it
+ * so that we have a recovery path if a writer aborts.
+ */
+ if (ClogCtl->page_number[slotno] != pageno ||
+ (ClogCtl->page_status[slotno] != CLOG_PAGE_DIRTY &&
+ ClogCtl->page_status[slotno] != CLOG_PAGE_WRITE_IN_PROGRESS))
+ {
+ S_UNLOCK(&(ClogCtl->buffer_lck[slotno]));
+ S_LOCK(&(ClogCtl->control_lck));
+ return;
+ }
+
+ /*
+ * Mark the slot write-busy. After this point, a transaction status
+ * update on this page will mark it dirty again. NB: we are assuming
+ * that read/write of the page status field is atomic, since we change
+ * the state while not holding control lock. However, we cannot set
+ * this state any sooner, or we'd possibly fool a previous writer
+ * into thinking he's successfully dumped the page when he hasn't.
+ * (Scenario: other writer starts, page is redirtied, we come along and
+ * set WRITE_IN_PROGRESS again, other writer completes and sets CLEAN
+ * because redirty info has been lost, then we think it's clean too.)
+ */
+ ClogCtl->page_status[slotno] = CLOG_PAGE_WRITE_IN_PROGRESS;
+
+ /* Okay, do the write */
+ CLOGPhysicalWritePage(pageno, slotno);
+
+ /* Re-acquire shared control lock and update page state */
+ S_LOCK(&(ClogCtl->control_lck));
+
+ Assert(ClogCtl->page_number[slotno] == pageno &&
+ (ClogCtl->page_status[slotno] == CLOG_PAGE_WRITE_IN_PROGRESS ||
+ ClogCtl->page_status[slotno] == CLOG_PAGE_DIRTY));
+
+ /* Cannot set CLEAN if someone re-dirtied page since write started */
+ if (ClogCtl->page_status[slotno] == CLOG_PAGE_WRITE_IN_PROGRESS)
+ ClogCtl->page_status[slotno] = CLOG_PAGE_CLEAN;
+
+ S_UNLOCK(&(ClogCtl->buffer_lck[slotno]));
+}
+
+/*
+ * Physical read of a (previously existing) page into a buffer slot
+ *
+ * For now, assume it's not worth keeping a file pointer open across
+ * read/write operations. We could cache one virtual file pointer ...
+ */
+static void
+CLOGPhysicalReadPage(int pageno, int slotno)
+{
+ int segno = pageno / CLOG_PAGES_PER_SEGMENT;
+ int rpageno = pageno % CLOG_PAGES_PER_SEGMENT;
+ int offset = rpageno * CLOG_BLCKSZ;
+ char path[MAXPGPATH];
+ int fd;
+
+ ClogFileName(path, segno);
+
+ /*
+ * In a crash-and-restart situation, it's possible for us to receive
+ * commands to set the commit status of transactions whose bits are
+ * in already-truncated segments of the commit log (see notes in
+ * CLOGPhysicalWritePage). Hence, if we are InRecovery, allow the
+ * case where the file doesn't exist, and return zeroes instead.
+ */
+ fd = BasicOpenFile(path, O_RDWR | PG_BINARY, S_IRUSR | S_IWUSR);
+ if (fd < 0)
+ {
+ if (errno != ENOENT || !InRecovery)
+ elog(STOP, "open of %s failed: %m", path);
+ elog(DEBUG, "clog file %s doesn't exist, reading as zeroes", path);
+ MemSet(ClogCtl->page_buffer[slotno], 0, CLOG_BLCKSZ);
+ return;
+ }
+
+ if (lseek(fd, (off_t) offset, SEEK_SET) < 0)
+ elog(STOP, "lseek of clog file %u, offset %u failed: %m",
+ segno, offset);
+
+ errno = 0;
+ if (read(fd, ClogCtl->page_buffer[slotno], CLOG_BLCKSZ) != CLOG_BLCKSZ)
+ elog(STOP, "read of clog file %u, offset %u failed: %m",
+ segno, offset);
+
+ close(fd);
+}
+
+/*
+ * Physical write of a page from a buffer slot
+ *
+ * For now, assume it's not worth keeping a file pointer open across
+ * read/write operations. We could cache one virtual file pointer ...
+ */
+static void
+CLOGPhysicalWritePage(int pageno, int slotno)
+{
+ int segno = pageno / CLOG_PAGES_PER_SEGMENT;
+ int rpageno = pageno % CLOG_PAGES_PER_SEGMENT;
+ int offset = rpageno * CLOG_BLCKSZ;
+ char path[MAXPGPATH];
+ int fd;
+
+ ClogFileName(path, segno);
+
+ /*
+ * If the file doesn't already exist, we should create it. It is possible
+ * for this to need to happen when writing a page that's not first in
+ * its segment; we assume the OS can cope with that. (Note: it might seem
+ * that it'd be okay to create files only when ZeroCLOGPage is called for
+ * the first page of a segment. However, if after a crash and restart
+ * the REDO logic elects to replay the log from a checkpoint before the
+ * latest one, then it's possible that we will get commands to set
+ * transaction status of transactions that have already been truncated
+ * from the commit log. Easiest way to deal with that is to accept
+ * references to nonexistent files here and in CLOGPhysicalReadPage.)
+ */
+ fd = BasicOpenFile(path, O_RDWR | PG_BINARY, S_IRUSR | S_IWUSR);
+ if (fd < 0)
+ {
+ if (errno != ENOENT)
+ elog(STOP, "open of %s failed: %m", path);
+ fd = BasicOpenFile(path, O_RDWR | O_CREAT | O_EXCL | PG_BINARY,
+ S_IRUSR | S_IWUSR);
+ if (fd < 0)
+ elog(STOP, "creation of file %s failed: %m", path);
+ }
+
+ if (lseek(fd, (off_t) offset, SEEK_SET) < 0)
+ elog(STOP, "lseek of clog file %u, offset %u failed: %m",
+ segno, offset);
+
+ errno = 0;
+ if (write(fd, ClogCtl->page_buffer[slotno], CLOG_BLCKSZ) != CLOG_BLCKSZ)
+ {
+ /* if write didn't set errno, assume problem is no disk space */
+ if (errno == 0)
+ errno = ENOSPC;
+ elog(STOP, "write of clog file %u, offset %u failed: %m",
+ segno, offset);
+ }
+
+ close(fd);
+}
+
+/*
+ * Select the slot to re-use when we need a free slot.
+ *
+ * The target page number is passed because we need to consider the
+ * possibility that some other process reads in the target page while
+ * we are doing I/O to free a slot. Hence, check or recheck to see if
+ * any slot already holds the target page, and return that slot if so.
+ * Thus, the returned slot is *either* a slot already holding the pageno
+ * (could be any state except EMPTY), *or* a freeable slot (state EMPTY
+ * or CLEAN).
+ *
+ * Control lock must be held at entry, and will be held at exit.
+ */
+static int
+SelectLRUCLOGPage(int pageno)
+{
+ /* Outer loop handles restart after I/O */
+ for (;;)
+ {
+ int slotno;
+ int bestslot = 0;
+ unsigned int bestcount = 0;
+
+ /* See if page already has a buffer assigned */
+ for (slotno = 0; slotno < NUM_CLOG_BUFFERS; slotno++)
+ {
+ if (ClogCtl->page_number[slotno] == pageno &&
+ ClogCtl->page_status[slotno] != CLOG_PAGE_EMPTY)
+ return slotno;
+ }
+
+ /*
+ * If we find any EMPTY slot, just select that one.
+ * Else locate the least-recently-used slot that isn't the
+ * latest CLOG page.
+ */
+ for (slotno = 0; slotno < NUM_CLOG_BUFFERS; slotno++)
+ {
+ if (ClogCtl->page_status[slotno] == CLOG_PAGE_EMPTY)
+ return slotno;
+ if (ClogCtl->page_lru_count[slotno] > bestcount &&
+ ClogCtl->page_number[slotno] != ClogCtl->latest_page_number)
+ {
+ bestslot = slotno;
+ bestcount = ClogCtl->page_lru_count[slotno];
+ }
+ }
+
+ /*
+ * If the selected page is clean, we're set.
+ */
+ if (ClogCtl->page_status[bestslot] == CLOG_PAGE_CLEAN)
+ return bestslot;
+
+ /*
+ * We need to do I/O. Normal case is that we have to write it out,
+ * but it's possible in the worst case to have selected a read-busy
+ * page. In that case we use ReadCLOGPage to wait for the read to
+ * complete.
+ */
+ if (ClogCtl->page_status[bestslot] == CLOG_PAGE_READ_IN_PROGRESS)
+ (void) ReadCLOGPage(ClogCtl->page_number[bestslot]);
+ else
+ WriteCLOGPage(bestslot);
+
+ /*
+ * Now loop back and try again. This is the easiest way of dealing
+ * with corner cases such as the victim page being re-dirtied while
+ * we wrote it.
+ */
+ }
+}
+
+/*
+ * This must be called ONCE during postmaster or standalone-backend startup,
+ * after StartupXLOG has initialized ShmemVariableCache->nextXid.
+ */
+void
+StartupCLOG(void)
+{
+ /*
+ * Initialize our idea of the latest page number.
+ */
+ ClogCtl->latest_page_number = TransactionIdToPage(ShmemVariableCache->nextXid);
+}
+
+/*
+ * This must be called ONCE during postmaster or standalone-backend shutdown
+ */
+void
+ShutdownCLOG(void)
+{
+ int slotno;
+
+ S_LOCK(&(ClogCtl->control_lck));
+
+ for (slotno = 0; slotno < NUM_CLOG_BUFFERS; slotno++)
+ {
+ WriteCLOGPage(slotno);
+ Assert(ClogCtl->page_status[slotno] == CLOG_PAGE_EMPTY ||
+ ClogCtl->page_status[slotno] == CLOG_PAGE_CLEAN);
+ }
+
+ S_UNLOCK(&(ClogCtl->control_lck));
+}
+
+/*
+ * Perform a checkpoint --- either during shutdown, or on-the-fly
+ */
+void
+CheckPointCLOG(void)
+{
+ int slotno;
+
+ S_LOCK(&(ClogCtl->control_lck));
+
+ for (slotno = 0; slotno < NUM_CLOG_BUFFERS; slotno++)
+ {
+ WriteCLOGPage(slotno);
+ /*
+ * We cannot assert that the slot is clean now, since another
+ * process might have re-dirtied it already. That's okay.
+ */
+ }
+
+ S_UNLOCK(&(ClogCtl->control_lck));
+}
+
+
+/*
+ * Make sure that CLOG has room for a newly-allocated XID.
+ *
+ * NB: this is called while holding XidGenLock. We want it to be very fast
+ * most of the time; even when it's not so fast, no actual I/O need happen
+ * unless we're forced to write out a dirty clog or xlog page to make room
+ * in shared memory.
+ */
+void
+ExtendCLOG(TransactionId newestXact)
+{
+ int pageno;
+
+ /* No work except at first XID of a page */
+ if (TransactionIdToPgIndex(newestXact) != 0)
+ return;
+
+ pageno = TransactionIdToPage(newestXact);
+
+ S_LOCK(&(ClogCtl->control_lck));
+
+ /* Zero the page and make an XLOG entry about it */
+ ZeroCLOGPage(pageno, true);
+
+ S_UNLOCK(&(ClogCtl->control_lck));
+}
+
+
+/*
+ * Remove all CLOG segments before the one holding the passed transaction ID
+ *
+ * When this is called, we know that the database logically contains no
+ * reference to transaction IDs older than oldestXact. However, we must
+ * not truncate the CLOG until we have performed a checkpoint, to ensure
+ * that no such references remain on disk either; else a crash just after
+ * the truncation might leave us with a problem. Since CLOG segments hold
+ * a large number of transactions, the opportunity to actually remove a
+ * segment is fairly rare, and so it seems best not to do the checkpoint
+ * unless we have confirmed that there is a removable segment. Therefore
+ * we issue the checkpoint command here, not in higher-level code as might
+ * seem cleaner.
+ */
+void
+TruncateCLOG(TransactionId oldestXact)
+{
+ int cutoffPage;
+ int slotno;
+
+ /*
+ * The cutoff point is the start of the segment containing oldestXact.
+ */
+ oldestXact -= oldestXact % CLOG_XACTS_PER_SEGMENT;
+ cutoffPage = TransactionIdToPage(oldestXact);
+
+ if (!ScanCLOGDirectory(cutoffPage, false))
+ return; /* nothing to remove */
+
+ /* Perform a CHECKPOINT */
+ CreateCheckPoint(false);
+
+ /*
+ * Scan CLOG shared memory and remove any pages preceding the cutoff
+ * page, to ensure we won't rewrite them later. (Any dirty pages
+ * should have been flushed already during the checkpoint, we're
+ * just being extra careful here.)
+ */
+ S_LOCK(&(ClogCtl->control_lck));
+
+restart:;
+
+ for (slotno = 0; slotno < NUM_CLOG_BUFFERS; slotno++)
+ {
+ if (ClogCtl->page_status[slotno] == CLOG_PAGE_EMPTY)
+ continue;
+ if (!CLOGPagePrecedes(ClogCtl->page_number[slotno], cutoffPage))
+ continue;
+ /*
+ * If page is CLEAN, just change state to EMPTY (expected case).
+ */
+ if (ClogCtl->page_status[slotno] == CLOG_PAGE_CLEAN)
+ {
+ ClogCtl->page_status[slotno] = CLOG_PAGE_EMPTY;
+ continue;
+ }
+ /*
+ * Hmm, we have (or may have) I/O operations acting on the page,
+ * so we've got to wait for them to finish and then start again.
+ * This is the same logic as in SelectLRUCLOGPage.
+ */
+ if (ClogCtl->page_status[slotno] == CLOG_PAGE_READ_IN_PROGRESS)
+ (void) ReadCLOGPage(ClogCtl->page_number[slotno]);
+ else
+ WriteCLOGPage(slotno);
+ goto restart;
+ }
+
+ S_UNLOCK(&(ClogCtl->control_lck));
+
+ /* Now we can remove the old CLOG segment(s) */
+ (void) ScanCLOGDirectory(cutoffPage, true);
+}
+
+/*
+ * TruncateCLOG subroutine: scan CLOG directory for removable segments.
+ * Actually remove them iff doDeletions is true. Return TRUE iff any
+ * removable segments were found. Note: no locking is needed.
+ */
+static bool
+ScanCLOGDirectory(int cutoffPage, bool doDeletions)
+{
+ bool found = false;
+ DIR *cldir;
+ struct dirent *clde;
+ int segno;
+ int segpage;
+ char path[MAXPGPATH];
+
+ cldir = opendir(ClogDir);
+ if (cldir == NULL)
+ elog(STOP, "could not open transaction-commit log directory (%s): %m",
+ ClogDir);
+
+ errno = 0;
+ while ((clde = readdir(cldir)) != NULL)
+ {
+ if (strlen(clde->d_name) == 4 &&
+ strspn(clde->d_name, "0123456789ABCDEF") == 4)
+ {
+ segno = (int) strtol(clde->d_name, NULL, 16);
+ segpage = segno * CLOG_PAGES_PER_SEGMENT;
+ if (CLOGPagePrecedes(segpage, cutoffPage))
+ {
+ found = true;
+ if (doDeletions)
+ {
+ snprintf(path, MAXPGPATH, "%s/%s", ClogDir, clde->d_name);
+ unlink(path);
+ }
+ }
+ }
+ errno = 0;
+ }
+ if (errno)
+ elog(STOP, "could not read transaction-commit log directory (%s): %m",
+ ClogDir);
+ closedir(cldir);
+
+ return found;
+}
+
+/*
+ * Decide which of two CLOG page numbers is "older" for truncation purposes.
+ *
+ * We need to use comparison of TransactionIds here in order to do the right
+ * thing with wraparound XID arithmetic. However, if we are asked about
+ * page number zero, we don't want to hand InvalidTransactionId to
+ * TransactionIdPrecedes: it'll get weird about permanent xact IDs. So,
+ * offset both xids by FirstNormalTransactionId to avoid that.
+ */
+static bool
+CLOGPagePrecedes(int page1, int page2)
+{
+ TransactionId xid1;
+ TransactionId xid2;
+
+ xid1 = (TransactionId) page1 * CLOG_XACTS_PER_PAGE;
+ xid1 += FirstNormalTransactionId;
+ xid2 = (TransactionId) page2 * CLOG_XACTS_PER_PAGE;
+ xid2 += FirstNormalTransactionId;
+
+ return TransactionIdPrecedes(xid1, xid2);
+}
+
+
+/*
+ * Write a ZEROPAGE xlog record
+ *
+ * Note: xlog record is marked as outside transaction control, since we
+ * want it to be redone whether the invoking transaction commits or not.
+ * (Besides which, this is normally done just before entering a transaction.)
+ */
+static void
+WriteZeroPageXlogRec(int pageno)
+{
+ XLogRecData rdata;
+
+ rdata.buffer = InvalidBuffer;
+ rdata.data = (char *) (&pageno);
+ rdata.len = sizeof(int);
+ rdata.next = NULL;
+ (void) XLogInsert(RM_CLOG_ID, CLOG_ZEROPAGE | XLOG_NO_TRAN, &rdata);
+}
+
+/*
+ * CLOG resource manager's routines
+ */
+void
+clog_redo(XLogRecPtr lsn, XLogRecord *record)
+{
+ uint8 info = record->xl_info & ~XLR_INFO_MASK;
+
+ if (info == CLOG_ZEROPAGE)
+ {
+ int pageno;
+ int slotno;
+
+ memcpy(&pageno, XLogRecGetData(record), sizeof(int));
+
+ S_LOCK(&(ClogCtl->control_lck));
+
+ slotno = ZeroCLOGPage(pageno, false);
+ WriteCLOGPage(slotno);
+ Assert(ClogCtl->page_status[slotno] == CLOG_PAGE_CLEAN);
+
+ S_UNLOCK(&(ClogCtl->control_lck));
+ }
+}
+
+void
+clog_undo(XLogRecPtr lsn, XLogRecord *record)
+{
+}
+
+void
+clog_desc(char *buf, uint8 xl_info, char *rec)
+{
+ uint8 info = xl_info & ~XLR_INFO_MASK;
+
+ if (info == CLOG_ZEROPAGE)
+ {
+ int pageno;
+
+ memcpy(&pageno, rec, sizeof(int));
+ sprintf(buf + strlen(buf), "zeropage: %d", pageno);
+ }
+ else
+ strcat(buf, "UNKNOWN");
+}
+/*
+ * rmgr.c
+ *
+ * Resource managers definition
+ *
+ * $Header: /cvsroot/pgsql/src/backend/access/transam/rmgr.c,v 1.9 2001/08/25 18:52:41 tgl Exp $
+ */
#include "postgres.h"
+
+#include "access/clog.h"
#include "access/gist.h"
#include "access/hash.h"
#include "access/heapam.h"
#include "storage/smgr.h"
#include "commands/sequence.h"
+
RmgrData RmgrTable[] = {
{"XLOG", xlog_redo, xlog_undo, xlog_desc},
{"Transaction", xact_redo, xact_undo, xact_desc},
{"Storage", smgr_redo, smgr_undo, smgr_desc},
- {"Reserved 3", NULL, NULL, NULL},
+ {"CLOG", clog_redo, clog_undo, clog_desc},
{"Reserved 4", NULL, NULL, NULL},
{"Reserved 5", NULL, NULL, NULL},
{"Reserved 6", NULL, NULL, NULL},
/*-------------------------------------------------------------------------
*
* transam.c
- * postgres transaction log/time interface routines
+ * postgres transaction log interface routines
*
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/access/transam/transam.c,v 1.46 2001/08/23 23:06:37 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/access/transam/transam.c,v 1.47 2001/08/25 18:52:41 tgl Exp $
*
* NOTES
* This file contains the high level access-method interface to the
#include "postgres.h"
-#include "access/heapam.h"
+#include "access/clog.h"
#include "access/transam.h"
-#include "catalog/catname.h"
-#include "miscadmin.h"
-static int RecoveryCheckingEnabled(void);
-static void TransRecover(Relation logRelation);
static bool TransactionLogTest(TransactionId transactionId, XidStatus status);
static void TransactionLogUpdate(TransactionId transactionId,
XidStatus status);
-/* ----------------
- * global variables holding pointers to relations used
- * by the transaction system. These are initialized by
- * InitializeTransactionLog().
- * ----------------
- */
-
-Relation LogRelation = (Relation) NULL;
-
/* ----------------
* Single-item cache for results of TransactionLogTest.
* ----------------
static TransactionId cachedTestXid = InvalidTransactionId;
static XidStatus cachedTestXidStatus;
-/* ----------------
- * transaction recovery state variables
- *
- * When the transaction system is initialized, we may
- * need to do recovery checking. This decision is decided
- * by the postmaster or the user by supplying the backend
- * with a special flag. In general, we want to do recovery
- * checking whenever we are running without a postmaster
- * or when the number of backends running under the postmaster
- * goes from zero to one. -cim 3/21/90
- * ----------------
- */
-static int RecoveryCheckingEnableState = 0;
-
-/* ----------------
- * recovery checking accessors
- * ----------------
- */
-static int
-RecoveryCheckingEnabled(void)
-{
- return RecoveryCheckingEnableState;
-}
-
-#ifdef NOT_USED
-static void
-SetRecoveryCheckingEnabled(bool state)
-{
- RecoveryCheckingEnableState = (state == true);
-}
-
-#endif
/* ----------------------------------------------------------------
* postgres log access method interface
*
* TransactionLogTest
* TransactionLogUpdate
- * ========
- * these functions do work for the interface
- * functions - they search/retrieve and append/update
- * information in the log and time relations.
* ----------------------------------------------------------------
*/
TransactionLogTest(TransactionId transactionId, /* transaction id to test */
XidStatus status) /* transaction status */
{
- BlockNumber blockNumber;
XidStatus xidstatus; /* recorded status of xid */
- bool fail = false; /* success/failure */
-
- /*
- * during initialization consider all transactions as having been
- * committed
- */
- if (!RelationIsValid(LogRelation))
- return (bool) (status == XID_COMMIT);
/*
- * before going to the buffer manager, check our single item cache to
+ * Before going to the commit log manager, check our single item cache to
* see if we didn't just check the transaction status a moment ago.
*/
if (TransactionIdEquals(transactionId, cachedTestXid))
- return (bool)
- (status == cachedTestXidStatus);
+ return (status == cachedTestXidStatus);
/*
- * compute the item pointer corresponding to the page containing our
- * transaction id. We save the item in our cache to speed up things
- * if we happen to ask for the same xid's status more than once.
+ * Also, check to see if the transaction ID is a permanent one.
*/
- TransComputeBlockNumber(LogRelation, transactionId, &blockNumber);
- xidstatus = TransBlockNumberGetXidStatus(LogRelation,
- blockNumber,
- transactionId,
- &fail);
-
- if (!fail)
+ if (! TransactionIdIsNormal(transactionId))
{
-
- /*
- * DO NOT cache status for transactions in unknown state !!!
- */
- if (xidstatus == XID_COMMIT || xidstatus == XID_ABORT)
- {
- TransactionIdStore(transactionId, &cachedTestXid);
- cachedTestXidStatus = xidstatus;
- }
- return (bool) (status == xidstatus);
+ if (TransactionIdEquals(transactionId, BootstrapTransactionId))
+ return (status == TRANSACTION_STATUS_COMMITTED);
+ if (TransactionIdEquals(transactionId, FrozenTransactionId))
+ return (status == TRANSACTION_STATUS_COMMITTED);
+ return (status == TRANSACTION_STATUS_ABORTED);
}
/*
- * here the block didn't contain the information we wanted
+ * Get the status.
*/
- elog(ERROR, "TransactionLogTest: failed to get xidstatus");
+ xidstatus = TransactionIdGetStatus(transactionId);
- /*
- * so lint is happy...
- */
- return false;
+ /*
+ * DO NOT cache status for unfinished transactions!
+ */
+ if (xidstatus != TRANSACTION_STATUS_IN_PROGRESS)
+ {
+ TransactionIdStore(transactionId, &cachedTestXid);
+ cachedTestXidStatus = xidstatus;
+ }
+
+ return (status == xidstatus);
}
/* --------------------------------
TransactionLogUpdate(TransactionId transactionId, /* trans id to update */
XidStatus status) /* new trans status */
{
- BlockNumber blockNumber;
- bool fail = false; /* success/failure */
-
/*
- * during initialization we don't record any updates.
+ * update the commit log
*/
- if (!RelationIsValid(LogRelation))
- return;
-
- /*
- * update the log relation
- */
- TransComputeBlockNumber(LogRelation, transactionId, &blockNumber);
- TransBlockNumberSetXidStatus(LogRelation,
- blockNumber,
- transactionId,
- status,
- &fail);
+ TransactionIdSetStatus(transactionId, status);
/*
* update (invalidate) our single item TransactionLogTest cache.
cachedTestXidStatus = status;
}
-/* ----------------------------------------------------------------
- * transaction recovery code
- * ----------------------------------------------------------------
- */
-
/* --------------------------------
- * TransRecover
- *
- * preform transaction recovery checking.
- *
- * Note: this should only be preformed if no other backends
- * are running. This is known by the postmaster and
- * conveyed by the postmaster passing a "do recovery checking"
- * flag to the backend.
- *
- * here we get the last recorded transaction from the log,
- * get the "last" and "next" transactions from the variable relation
- * and then preform some integrity tests:
- *
- * 1) No transaction may exist higher then the "next" available
- * transaction recorded in the variable relation. If this is the
- * case then it means either the log or the variable relation
- * has become corrupted.
- *
- * 2) The last committed transaction may not be higher then the
- * next available transaction for the same reason.
- *
- * 3) The last recorded transaction may not be lower then the
- * last committed transaction. (the reverse is ok - it means
- * that some transactions have aborted since the last commit)
- *
- * Here is what the proper situation looks like. The line
- * represents the data stored in the log. 'c' indicates the
- * transaction was recorded as committed, 'a' indicates an
- * abortted transaction and '.' represents information not
- * recorded. These may correspond to in progress transactions.
- *
- * c c a c . . a . . . . . . . . . .
- * | |
- * last next
- *
- * Since "next" is only incremented by GetNewTransactionId() which
- * is called when transactions are started. Hence if there
- * are commits or aborts after "next", then it means we committed
- * or aborted BEFORE we started the transaction. This is the
- * rational behind constraint (1).
- *
- * Likewise, "last" should never greater then "next" for essentially
- * the same reason - it would imply we committed before we started.
- * This is the reasoning for (2).
- *
- * (3) implies we may never have a situation such as:
+ * AmiTransactionOverride
*
- * c c a c . . a c . . . . . . . . .
- * | |
- * last next
- *
- * where there is a 'c' greater then "last".
- *
- * Recovery checking is more difficult in the case where
- * several backends are executing concurrently because the
- * transactions may be executing in the other backends.
- * So, we only do recovery stuff when the backend is explicitly
- * passed a flag on the command line.
+ * This function is used to manipulate the bootstrap flag.
* --------------------------------
*/
-static void
-TransRecover(Relation logRelation)
+void
+AmiTransactionOverride(bool flag)
{
+ AMI_OVERRIDE = flag;
}
/* ----------------------------------------------------------------
* Interface functions
*
- * InitializeTransactionLog
- * ========
- * this function (called near cinit) initializes
- * the transaction log, time and variable relations.
- *
* TransactionId DidCommit
* TransactionId DidAbort
* TransactionId IsInProgress
*
* TransactionId Commit
* TransactionId Abort
- * TransactionId SetInProgress
* ========
* these functions set the transaction status
- * of the specified xid. TransactionIdCommit() also
- * records the current time in the time relation
- * and updates the variable relation counter.
+ * of the specified xid.
*
* ----------------------------------------------------------------
*/
-/*
- * InitializeTransactionLog
- * Initializes transaction logging.
- */
-void
-InitializeTransactionLog(void)
-{
- Relation logRelation;
- MemoryContext oldContext;
-
- /*
- * don't do anything during bootstrapping
- */
- if (AMI_OVERRIDE)
- return;
-
- /*
- * disable the transaction system so the access methods don't
- * interfere during initialization.
- */
- OverrideTransactionSystem(true);
-
- /*
- * make sure allocations occur within the top memory context so that
- * our log management structures are protected from garbage collection
- * at the end of every transaction.
- */
- oldContext = MemoryContextSwitchTo(TopMemoryContext);
-
- /*
- * first open the log and time relations (these are created by amiint
- * so they are guaranteed to exist)
- */
- logRelation = heap_openr(LogRelationName, NoLock);
-
- /*
- * XXX TransactionLogUpdate requires that LogRelation is valid so we
- * temporarily set it so we can initialize things properly. This could
- * be done cleaner.
- */
- LogRelation = logRelation;
-
- /*
- * if we have a virgin database, we initialize the log relation by
- * committing the BootstrapTransactionId and we initialize the
- * variable relation by setting the next available transaction id to
- * FirstNormalTransactionId. OID initialization happens as a side
- * effect of bootstrapping in varsup.c.
- */
- SpinAcquire(OidGenLockId);
- if (!TransactionIdDidCommit(BootstrapTransactionId))
- {
- TransactionLogUpdate(BootstrapTransactionId, XID_COMMIT);
- Assert(!IsUnderPostmaster &&
- TransactionIdEquals(ShmemVariableCache->nextXid,
- FirstNormalTransactionId));
- ShmemVariableCache->nextXid = FirstNormalTransactionId;
- }
- else if (RecoveryCheckingEnabled())
- {
-
- /*
- * if we have a pre-initialized database and if the perform
- * recovery checking flag was passed then we do our database
- * integrity checking.
- */
- TransRecover(logRelation);
- }
- LogRelation = (Relation) NULL;
- SpinRelease(OidGenLockId);
-
- /*
- * now re-enable the transaction system
- */
- OverrideTransactionSystem(false);
-
- /*
- * instantiate the global variables
- */
- LogRelation = logRelation;
-
- /*
- * restore the memory context to the previous context before we return
- * from initialization.
- */
- MemoryContextSwitchTo(oldContext);
-}
-
/* --------------------------------
* TransactionId DidCommit
* TransactionId DidAbort
if (AMI_OVERRIDE)
return true;
- return TransactionLogTest(transactionId, XID_COMMIT);
+ return TransactionLogTest(transactionId, TRANSACTION_STATUS_COMMITTED);
}
/*
- * TransactionIdDidAborted
+ * TransactionIdDidAbort
* True iff transaction associated with the identifier did abort.
*
* Note:
* Assumes transaction identifier is valid.
- * XXX Is this unneeded?
*/
bool /* true if given transaction aborted */
TransactionIdDidAbort(TransactionId transactionId)
if (AMI_OVERRIDE)
return false;
- return TransactionLogTest(transactionId, XID_ABORT);
+ return TransactionLogTest(transactionId, TRANSACTION_STATUS_ABORTED);
}
/*
* PROC structures of all running backend. - vadim 11/26/96
*
* Old comments:
- * true if given transaction neither committed nor aborted
-
+ * true if given transaction has neither committed nor aborted
+ */
+#ifdef NOT_USED
bool
TransactionIdIsInProgress(TransactionId transactionId)
{
if (AMI_OVERRIDE)
return false;
- return TransactionLogTest(transactionId, XID_INPROGRESS);
+ return TransactionLogTest(transactionId, TRANSACTION_STATUS_IN_PROGRESS);
}
- */
+#endif /* NOT_USED */
/* --------------------------------
* TransactionId Commit
* TransactionId Abort
- * TransactionId SetInProgress
* --------------------------------
*/
if (AMI_OVERRIDE)
return;
- TransactionLogUpdate(transactionId, XID_COMMIT);
+ TransactionLogUpdate(transactionId, TRANSACTION_STATUS_COMMITTED);
}
/*
if (AMI_OVERRIDE)
return;
- TransactionLogUpdate(transactionId, XID_ABORT);
+ TransactionLogUpdate(transactionId, TRANSACTION_STATUS_ABORTED);
}
+++ /dev/null
-/*-------------------------------------------------------------------------
- *
- * transsup.c
- * postgres transaction access method support code
- *
- * Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
- * Portions Copyright (c) 1994, Regents of the University of California
- *
- *
- * IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/access/transam/Attic/transsup.c,v 1.31 2001/07/12 04:11:13 tgl Exp $
- *
- * NOTES
- * This file contains support functions for the high
- * level access method interface routines found in transam.c
- *
- *-------------------------------------------------------------------------
- */
-#include "postgres.h"
-
-#include "access/xact.h"
-#include "utils/bit.h"
-
-
-/* ----------------
- * transaction system version id
- *
- * this is stored on the first page of the log, time and variable
- * relations on the first 4 bytes. This is so that if we improve
- * the format of the transaction log after postgres version 2, then
- * people won't have to rebuild their databases.
- *
- * TRANS_SYSTEM_VERSION 100 means major version 1 minor version 0.
- * Two databases with the same major version should be compatible,
- * even if their minor versions differ.
- *
- * XXX This isn't actually being used!
- * ----------------
- */
-#define TRANS_SYSTEM_VERSION 200
-
-/* ----------------
- * LogRelationContents structure
- *
- * This structure describes the storage of the data in the
- * first 128 bytes of the log relation. This storage is never
- * used for transaction status because transaction id's begin
- * their numbering at 512.
- *
- * The first 4 bytes of this relation store the version
- * number of the transaction system.
- *
- * XXX This isn't actually being used!
- * ----------------
- */
-typedef struct LogRelationContentsData
-{
- XLogRecPtr LSN; /* temp hack: LSN is member of any block */
- /* so should be described in bufmgr */
- int TransSystemVersion;
-} LogRelationContentsData;
-
-typedef LogRelationContentsData *LogRelationContents;
-
-
-/* ----------------
- * BitIndexOf computes the index of the Nth xid on a given block
- * ----------------
- */
-#define BitIndexOf(N) ((N) * 2)
-
-/* ----------------
- * transaction page definitions
- * ----------------
- */
-#define TP_DataSize (BLCKSZ - sizeof(XLogRecPtr))
-#define TP_NumXidStatusPerBlock (TP_DataSize * 4)
-
-
-static XidStatus TransBlockGetXidStatus(Block tblock,
- TransactionId transactionId);
-static void TransBlockSetXidStatus(Block tblock,
- TransactionId transactionId, XidStatus xstatus);
-
-/* ----------------------------------------------------------------
- * general support routines
- * ----------------------------------------------------------------
- */
-
-/* --------------------------------
- * AmiTransactionOverride
- *
- * This function is used to manipulate the bootstrap flag.
- * --------------------------------
- */
-void
-AmiTransactionOverride(bool flag)
-{
- AMI_OVERRIDE = flag;
-}
-
-/* --------------------------------
- * TransComputeBlockNumber
- * --------------------------------
- */
-void
-TransComputeBlockNumber(Relation relation, /* relation to test */
- TransactionId transactionId, /* transaction id to
- * test */
- BlockNumber *blockNumberOutP)
-{
- uint32 itemsPerBlock = 0;
-
- /*
- * we calculate the block number of our transaction by dividing the
- * transaction id by the number of transaction things per block.
- */
- if (relation == LogRelation)
- itemsPerBlock = TP_NumXidStatusPerBlock;
- else
- elog(ERROR, "TransComputeBlockNumber: unknown relation");
-
- /* ----------------
- * warning! if the transaction id's get too large
- * then a BlockNumber may not be large enough to hold the results
- * of our division.
- *
- * XXX this will all vanish soon when we implement an improved
- * transaction id schema -cim 3/23/90
- *
- * This has vanished now that xid's are 4 bytes (no longer 5).
- * -mer 5/24/92
- * ----------------
- */
- (*blockNumberOutP) = transactionId / itemsPerBlock;
-}
-
-
-/* ----------------------------------------------------------------
- * trans block support routines
- * ----------------------------------------------------------------
- */
-
-/* --------------------------------
- * TransBlockGetLastTransactionIdStatus
- *
- * This returns the status and transaction id of the last
- * transaction information recorded on the given TransBlock.
- * --------------------------------
- */
-
-#ifdef NOT_USED
-static XidStatus
-TransBlockGetLastTransactionIdStatus(Block tblock,
- TransactionId baseXid,
- TransactionId *returnXidP)
-{
- Index index;
- Index maxIndex;
- bits8 bit1;
- bits8 bit2;
- BitIndex offset;
- XidStatus xstatus;
-
- /*
- * sanity check
- */
- Assert((tblock != NULL));
-
- /*
- * search downward from the top of the block data, looking for the
- * first Non-in progress transaction status. Since we are scanning
- * backward, this will be last recorded transaction status on the
- * block.
- */
- maxIndex = TP_NumXidStatusPerBlock;
- for (index = maxIndex; index > 0; index--)
- {
- offset = BitIndexOf(index - 1);
- bit1 = ((bits8) BitArrayBitIsSet((BitArray) tblock, offset++)) << 1;
- bit2 = (bits8) BitArrayBitIsSet((BitArray) tblock, offset);
-
- xstatus = (bit1 | bit2);
-
- /*
- * here we have the status of some transaction, so test if the
- * status is recorded as "in progress". If so, then we save the
- * transaction id in the place specified by the caller.
- */
- if (xstatus != XID_INPROGRESS)
- {
- if (returnXidP != NULL)
- TransactionIdStore(baseXid + (index - 1), returnXidP);
- break;
- }
- }
-
- /*
- * if we get here and index is 0 it means we couldn't find a
- * non-inprogress transaction on the block. For now we just return
- * this info to the user. They can check if the return status is "in
- * progress" to know this condition has arisen.
- */
- if (index == 0)
- {
- if (returnXidP != NULL)
- TransactionIdStore(baseXid, returnXidP);
- }
-
- /*
- * return the status to the user
- */
- return xstatus;
-}
-
-#endif
-
-/* --------------------------------
- * TransBlockGetXidStatus
- *
- * This returns the status of the desired transaction
- * --------------------------------
- */
-
-static XidStatus
-TransBlockGetXidStatus(Block tblock,
- TransactionId transactionId)
-{
- Index index;
- bits8 bit1;
- bits8 bit2;
- BitIndex offset;
-
- tblock = (Block) ((char *) tblock + sizeof(XLogRecPtr));
-
- /* ----------------
- * calculate the index into the transaction data where
- * our transaction status is located
- *
- * XXX this will be replaced soon when we move to the
- * new transaction id scheme -cim 3/23/90
- *
- * The old system has now been replaced. -mer 5/24/92
- * ----------------
- */
- index = transactionId % TP_NumXidStatusPerBlock;
-
- /*
- * get the data at the specified index
- */
- offset = BitIndexOf(index);
- bit1 = ((bits8) BitArrayBitIsSet((BitArray) tblock, offset++)) << 1;
- bit2 = (bits8) BitArrayBitIsSet((BitArray) tblock, offset);
-
- /*
- * return the transaction status to the caller
- */
- return (XidStatus) (bit1 | bit2);
-}
-
-/* --------------------------------
- * TransBlockSetXidStatus
- *
- * This sets the status of the desired transaction
- * --------------------------------
- */
-static void
-TransBlockSetXidStatus(Block tblock,
- TransactionId transactionId,
- XidStatus xstatus)
-{
- Index index;
- BitIndex offset;
-
- tblock = (Block) ((char *) tblock + sizeof(XLogRecPtr));
-
- /* ----------------
- * calculate the index into the transaction data where
- * we sould store our transaction status.
- *
- * XXX this will be replaced soon when we move to the
- * new transaction id scheme -cim 3/23/90
- *
- * The new scheme is here -mer 5/24/92
- * ----------------
- */
- index = transactionId % TP_NumXidStatusPerBlock;
-
- offset = BitIndexOf(index);
-
- /*
- * store the transaction value at the specified offset
- */
- switch (xstatus)
- {
- case XID_COMMIT: /* set 10 */
- BitArraySetBit((BitArray) tblock, offset);
- BitArrayClearBit((BitArray) tblock, offset + 1);
- break;
- case XID_ABORT: /* set 01 */
- BitArrayClearBit((BitArray) tblock, offset);
- BitArraySetBit((BitArray) tblock, offset + 1);
- break;
- case XID_INPROGRESS: /* set 00 */
- BitArrayClearBit((BitArray) tblock, offset);
- BitArrayClearBit((BitArray) tblock, offset + 1);
- break;
- default:
- elog(NOTICE,
- "TransBlockSetXidStatus: invalid status: %d (ignored)",
- xstatus);
- break;
- }
-}
-
-/* ----------------------------------------------------------------
- * transam i/o support routines
- * ----------------------------------------------------------------
- */
-
-/* --------------------------------
- * TransBlockNumberGetXidStatus
- * --------------------------------
- */
-XidStatus
-TransBlockNumberGetXidStatus(Relation relation,
- BlockNumber blockNumber,
- TransactionId xid,
- bool *failP)
-{
- Buffer buffer; /* buffer associated with block */
- Block block; /* block containing xstatus */
- XidStatus xstatus; /* recorded status of xid */
- bool localfail; /* bool used if failP = NULL */
-
- /*
- * get the page containing the transaction information
- */
- buffer = ReadBuffer(relation, blockNumber);
- LockBuffer(buffer, BUFFER_LOCK_SHARE);
- block = BufferGetBlock(buffer);
-
- /*
- * get the status from the block. note, for now we always return
- * false in failP.
- */
- if (failP == NULL)
- failP = &localfail;
- (*failP) = false;
-
- xstatus = TransBlockGetXidStatus(block, xid);
-
- /*
- * release the buffer and return the status
- */
- LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
- ReleaseBuffer(buffer);
-
- return xstatus;
-}
-
-/* --------------------------------
- * TransBlockNumberSetXidStatus
- * --------------------------------
- */
-void
-TransBlockNumberSetXidStatus(Relation relation,
- BlockNumber blockNumber,
- TransactionId xid,
- XidStatus xstatus,
- bool *failP)
-{
- Buffer buffer; /* buffer associated with block */
- Block block; /* block containing xstatus */
- bool localfail; /* bool used if failP = NULL */
-
- /*
- * get the block containing the transaction status
- */
- buffer = ReadBuffer(relation, blockNumber);
- LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
- block = BufferGetBlock(buffer);
-
- /*
- * attempt to update the status of the transaction on the block. if we
- * are successful, write the block. otherwise release the buffer.
- * note, for now we always return false in failP.
- */
- if (failP == NULL)
- failP = &localfail;
- (*failP) = false;
-
- TransBlockSetXidStatus(block, xid, xstatus);
-
- LockBuffer(buffer, BUFFER_LOCK_UNLOCK);
- if ((*failP) == false)
- WriteBuffer(buffer);
- else
- ReleaseBuffer(buffer);
-}
-
-/* --------------------------------
- * TransGetLastRecordedTransaction
- * --------------------------------
- */
-#ifdef NOT_USED
-void
-TransGetLastRecordedTransaction(Relation relation,
- TransactionId xid, /* return: transaction
- * id */
- bool *failP)
-{
- BlockNumber blockNumber; /* block number */
- Buffer buffer; /* buffer associated with block */
- Block block; /* block containing xid status */
- BlockNumber n; /* number of blocks in the relation */
- TransactionId baseXid;
-
- (*failP) = false;
-
- /*
- * SOMEDAY gain exclusive access to the log relation
- *
- * That someday is today 5 Aug. 1991 -mer It looks to me like we only
- * need to set a read lock here, despite the above comment about
- * exclusive access. The block is never actually written into, we
- * only check status bits.
- */
- RelationSetLockForRead(relation);
-
- /*
- * we assume the last block of the log contains the last recorded
- * transaction. If the relation is empty we return failure to the
- * user.
- */
- n = RelationGetNumberOfBlocks(relation);
- if (n == 0)
- {
- (*failP) = true;
- return;
- }
-
- /*
- * get the block containing the transaction information
- */
- blockNumber = n - 1;
- buffer = ReadBuffer(relation, blockNumber);
- block = BufferGetBlock(buffer);
-
- /*
- * get the last xid on the block
- */
- baseXid = blockNumber * TP_NumXidStatusPerBlock;
-
-/* XXX ???? xid won't get returned! - AY '94 */
- TransBlockGetLastTransactionIdStatus(block, baseXid, &xid);
-
- ReleaseBuffer(buffer);
-
- /*
- * SOMEDAY release our lock on the log relation
- */
- RelationUnsetLockForRead(relation);
-}
-
-#endif
* Copyright (c) 2000, PostgreSQL Global Development Group
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/access/transam/varsup.c,v 1.44 2001/08/23 23:06:37 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/access/transam/varsup.c,v 1.45 2001/08/25 18:52:41 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
+#include "access/clog.h"
#include "access/transam.h"
-#include "access/xlog.h"
#include "storage/proc.h"
/*
* Allocate the next XID for my new transaction.
*/
-void
-GetNewTransactionId(TransactionId *xid)
+TransactionId
+GetNewTransactionId(void)
{
+ TransactionId xid;
+
/*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
if (AMI_OVERRIDE)
- {
- *xid = BootstrapTransactionId;
- return;
- }
+ return BootstrapTransactionId;
SpinAcquire(XidGenLockId);
- *xid = ShmemVariableCache->nextXid;
+ xid = ShmemVariableCache->nextXid;
TransactionIdAdvance(ShmemVariableCache->nextXid);
+ /*
+ * If we have just allocated the first XID of a new page of the
+ * commit log, zero out that commit-log page before returning.
+ * We must do this while holding XidGenLock, else another xact could
+ * acquire and commit a later XID before we zero the page. Fortunately,
+ * a page of the commit log holds 32K or more transactions, so we don't
+ * have to do this very often.
+ */
+ ExtendCLOG(xid);
+
/*
* Must set MyProc->xid before releasing XidGenLock. This ensures that
* when GetSnapshotData calls ReadNewTransactionId, all active XIDs
* removed while holding the lock.)
*/
if (MyProc != (PROC *) NULL)
- MyProc->xid = *xid;
+ MyProc->xid = xid;
SpinRelease(XidGenLockId);
+
+ return xid;
}
/*
* Read nextXid but don't allocate it.
*/
-void
-ReadNewTransactionId(TransactionId *xid)
+TransactionId
+ReadNewTransactionId(void)
{
+ TransactionId xid;
+
/*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
if (AMI_OVERRIDE)
- {
- *xid = BootstrapTransactionId;
- return;
- }
+ return BootstrapTransactionId;
SpinAcquire(XidGenLockId);
- *xid = ShmemVariableCache->nextXid;
+ xid = ShmemVariableCache->nextXid;
SpinRelease(XidGenLockId);
+
+ return xid;
}
/* ----------------------------------------------------------------
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/access/transam/xact.c,v 1.108 2001/07/16 22:43:33 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/access/transam/xact.c,v 1.109 2001/08/25 18:52:41 tgl Exp $
*
* NOTES
* Transaction aborts can now occur two ways:
static void (*_RollbackFunc) (void *) = NULL;
static void *_RollbackData = NULL;
-/* ----------------
- * info returned when the system is disabled
- *
- * Apparently a lot of this code is inherited from other prototype systems.
- *
- * For DisabledStartTime, use a symbolic value to make the relationships clearer.
- * The old value of 1073741823 corresponds to a date in y2004, which is coming closer
- * every day. It appears that if we return a value guaranteed larger than
- * any real time associated with a transaction then comparisons in other
- * modules will still be correct. Let's use BIG_ABSTIME for this. tgl 2/14/97
- * ----------------
- */
-static CommandId DisabledCommandId = (CommandId) -1;
-
-static AbsoluteTime DisabledStartTime = (AbsoluteTime) BIG_ABSTIME;
-
/* ----------------
* catalog creation transaction bootstrapping flag.
* This should be eliminated and added to the transaction
return true;
case TRANS_ABORT:
return true;
- case TRANS_DISABLED:
- return false;
}
/*
return false;
}
-/* --------------------------------
- * OverrideTransactionSystem
- *
- * This is used to temporarily disable the transaction
- * processing system in order to do initialization of
- * the transaction system data structures and relations
- * themselves.
- * --------------------------------
- */
-static int SavedTransactionState;
-
-void
-OverrideTransactionSystem(bool flag)
-{
- TransactionState s = CurrentTransactionState;
-
- if (flag == true)
- {
- if (s->state == TRANS_DISABLED)
- return;
-
- SavedTransactionState = s->state;
- s->state = TRANS_DISABLED;
- }
- else
- {
- if (s->state != TRANS_DISABLED)
- return;
-
- s->state = SavedTransactionState;
- }
-}
/* --------------------------------
* GetCurrentTransactionId
- *
- * This returns the id of the current transaction, or
- * the id of the "disabled" transaction.
* --------------------------------
*/
TransactionId
{
TransactionState s = CurrentTransactionState;
- /*
- * if the transaction system is disabled, we return the special
- * "disabled" transaction id.
- */
- if (s->state == TRANS_DISABLED)
- return DisabledTransactionId;
-
- /*
- * otherwise return the current transaction id.
- */
return s->transactionIdData;
}
{
TransactionState s = CurrentTransactionState;
- /*
- * if the transaction system is disabled, we return the special
- * "disabled" command id.
- */
- if (s->state == TRANS_DISABLED)
- return DisabledCommandId;
-
return s->commandId;
}
{
TransactionState s = CurrentTransactionState;
- /*
- * if the transaction system is disabled, we return the special
- * "disabled" command id.
- */
- if (s->state == TRANS_DISABLED)
- return DisabledCommandId;
-
return s->scanCommandId;
}
{
TransactionState s = CurrentTransactionState;
- /*
- * if the transaction system is disabled, we return the special
- * "disabled" starting time.
- */
- if (s->state == TRANS_DISABLED)
- return DisabledStartTime;
-
return s->startTime;
}
CurrentTransactionStateData.scanCommandId = savedId;
}
-/* ----------------------------------------------------------------
- * initialization stuff
- * ----------------------------------------------------------------
- */
-void
-InitializeTransactionSystem(void)
-{
- InitializeTransactionLog();
-}
-
/* ----------------------------------------------------------------
* StartTransaction stuff
* ----------------------------------------------------------------
* --------------------------------
*/
void
-RecordTransactionCommit()
+RecordTransactionCommit(void)
{
TransactionId xid;
bool leak;
- xid = GetCurrentTransactionId();
-
leak = BufferPoolCheckLeak();
+ xid = GetCurrentTransactionId();
+
+ /*
+ * We needn't write anything in xlog or clog if the transaction was
+ * read-only, which we check by testing if it made any xlog entries.
+ */
if (MyLastRecPtr.xrecoff != 0)
{
XLogRecData rdata;
/* Break the chain of back-links in the XLOG records I output */
MyLastRecPtr.xrecoff = 0;
+ /* Mark the transaction committed in clog */
TransactionIdCommit(xid);
END_CRIT_SECTION();
TransactionId xid = GetCurrentTransactionId();
/*
- * Double check here is to catch case that we aborted partway through
+ * We needn't write anything in xlog or clog if the transaction was
+ * read-only, which we check by testing if it made any xlog entries.
+ *
+ * Extra check here is to catch case that we aborted partway through
* RecordTransactionCommit ...
*/
if (MyLastRecPtr.xrecoff != 0 && !TransactionIdDidCommit(xid))
START_CRIT_SECTION();
+ /*
+ * SHOULD SAVE ARRAY OF RELFILENODE-s TO DROP
+ */
recptr = XLogInsert(RM_XACT_ID, XLOG_XACT_ABORT, &rdata);
+ /*
+ * There's no need for XLogFlush here, since the default assumption
+ * would be that we aborted, anyway.
+ */
+
+ /* Mark the transaction aborted in clog */
TransactionIdAbort(xid);
END_CRIT_SECTION();
* fix to a communications problem, and we keep having to deal with it
* here. We should fix the comm channel code. mao 080891
*/
- if (s->state == TRANS_DISABLED || s->state == TRANS_INPROGRESS)
+ if (s->state == TRANS_INPROGRESS)
return;
/*
/*
* generate a new transaction id
*/
- GetNewTransactionId(&(s->transactionIdData));
+ s->transactionIdData = GetNewTransactionId();
XactLockTableInsert(s->transactionIdData);
/*
* check the current transaction state
*/
- if (s->state == TRANS_DISABLED)
- return;
-
if (s->state != TRANS_INPROGRESS)
- elog(NOTICE, "CommitTransaction and not in in-progress state ");
+ elog(NOTICE, "CommitTransaction and not in in-progress state");
/*
* Tell the trigger manager that this transaction is about to be
/*
* check the current transaction state
*/
- if (s->state == TRANS_DISABLED)
- {
- RESUME_INTERRUPTS();
- return;
- }
-
if (s->state != TRANS_INPROGRESS)
elog(NOTICE, "AbortTransaction and not in in-progress state");
CloseSequences();
AtEOXact_portals();
- /* Advertise the fact that we aborted in pg_log. */
+ /* Advertise the fact that we aborted in pg_clog. */
RecordTransactionAbort();
/*
{
TransactionState s = CurrentTransactionState;
- if (s->state == TRANS_DISABLED)
- return;
-
/*
* State should still be TRANS_ABORT from AbortTransaction().
*/
/*
* check the current transaction state
*/
- if (s->state == TRANS_DISABLED)
- return;
-
if (s->blockState != TBLOCK_DEFAULT)
elog(NOTICE, "BEGIN: already a transaction in progress");
/*
* check the current transaction state
*/
- if (s->state == TRANS_DISABLED)
- return;
-
if (s->blockState == TBLOCK_INPROGRESS)
{
/*
* check the current transaction state
*/
- if (s->state == TRANS_DISABLED)
- return;
-
if (s->blockState == TBLOCK_INPROGRESS)
{
{
TransactionState s = CurrentTransactionState;
- /*
- * check the current transaction state
- */
- if (s->state == TRANS_DISABLED)
- return;
-
/*
* if the transaction has already been automatically aborted with an
* error, and the user subsequently types 'abort', allow it. (the
CleanupTransaction();
break;
case TRANS_DEFAULT:
- case TRANS_DISABLED:
/* Not in a transaction, do nothing */
break;
}
/* SHOULD REMOVE FILES OF ALL DROPPED RELATIONS */
}
else if (info == XLOG_XACT_ABORT)
+ {
TransactionIdAbort(record->xl_xid);
+ /* SHOULD REMOVE FILES OF ALL FAILED-TO-BE-CREATED RELATIONS */
+ }
else
elog(STOP, "xact_redo: unknown op code %u", info);
}
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Header: /cvsroot/pgsql/src/backend/access/transam/xlog.c,v 1.74 2001/08/23 23:06:37 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/access/transam/xlog.c,v 1.75 2001/08/25 18:52:41 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include <locale.h>
#endif
+#include "access/clog.h"
#include "access/transam.h"
#include "access/xact.h"
+#include "access/xlog.h"
+#include "access/xlogutils.h"
#include "catalog/catversion.h"
#include "catalog/pg_control.h"
#include "storage/sinval.h"
#include "storage/spin.h"
#include "storage/s_lock.h"
#include "storage/bufpage.h"
-#include "access/xlog.h"
-#include "access/xlogutils.h"
#include "utils/builtins.h"
#include "utils/relcache.h"
#include "utils/selfuncs.h"
strspn(xlde->d_name, "0123456789ABCDEF") == 16 &&
strcmp(xlde->d_name, lastoff) <= 0)
{
- sprintf(path, "%s/%s", XLogDir, xlde->d_name);
+ snprintf(path, MAXPGPATH, "%s/%s", XLogDir, xlde->d_name);
if (XLOG_archive_dir[0])
{
elog(LOG, "archiving transaction log file %s",
/* some additional ControlFile fields are set in WriteControlFile() */
WriteControlFile();
+
+ /* Bootstrap the commit log, too */
+ BootStrapCLOG();
}
static char *
ControlFile->time = time(NULL);
UpdateControlFile();
- XLogOpenLogRelation(); /* open pg_log */
XLogInitRelationCache();
/* Is REDO required ? */
ThisStartUpID++;
XLogCtl->ThisStartUpID = ThisStartUpID;
+ /* Start up the commit log, too */
+ StartupCLOG();
+
elog(LOG, "database system is ready");
CritSectionCount--;
CritSectionCount++;
CreateDummyCaches();
CreateCheckPoint(true);
+ ShutdownCLOG();
CritSectionCount--;
elog(LOG, "database system is shut down");
*/
FlushBufferPool();
+ /* And commit-log buffers, too */
+ CheckPointCLOG();
+
/*
* Now insert the checkpoint record into XLOG.
*/
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Header: /cvsroot/pgsql/src/backend/access/transam/xlogutils.c,v 1.17 2001/08/23 23:06:37 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/access/transam/xlogutils.c,v 1.18 2001/08/25 18:52:41 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
-#include "access/xlog.h"
-#include "access/transam.h"
-#include "access/xact.h"
-#include "storage/bufpage.h"
-#include "storage/bufmgr.h"
-#include "storage/smgr.h"
#include "access/htup.h"
#include "access/xlogutils.h"
#include "catalog/pg_database.h"
#include "lib/hasht.h"
+#include "storage/bufpage.h"
+#include "storage/smgr.h"
#include "utils/relcache.h"
+
/*
* ---------------------------------------------------------------
*
return (true);
}
-/*
- * Open pg_log in recovery
- */
-extern Relation LogRelation; /* pg_log relation */
-
-void
-XLogOpenLogRelation(void)
-{
- Relation logRelation;
-
- Assert(!LogRelation);
- logRelation = (Relation) malloc(sizeof(RelationData));
- memset(logRelation, 0, sizeof(RelationData));
- logRelation->rd_rel = (Form_pg_class) malloc(sizeof(FormData_pg_class));
- memset(logRelation->rd_rel, 0, sizeof(FormData_pg_class));
-
- sprintf(RelationGetPhysicalRelationName(logRelation), "pg_log");
- logRelation->rd_node.tblNode = InvalidOid;
- logRelation->rd_node.relNode = RelOid_pg_log;
- logRelation->rd_targblock = InvalidBlockNumber;
- logRelation->rd_fd = -1;
- logRelation->rd_fd = smgropen(DEFAULT_SMGR, logRelation, false);
- if (logRelation->rd_fd < 0)
- elog(STOP, "XLogOpenLogRelation: failed to open pg_log");
- LogRelation = logRelation;
-}
-
/*
* ---------------------------------------------------------------
*
#
# Makefile for catalog
#
-# $Header: /cvsroot/pgsql/src/backend/catalog/Makefile,v 1.36 2001/08/24 14:07:48 petere Exp $
+# $Header: /cvsroot/pgsql/src/backend/catalog/Makefile,v 1.37 2001/08/25 18:52:41 tgl Exp $
#
#-------------------------------------------------------------------------
pg_operator.h pg_opclass.h pg_am.h pg_amop.h pg_amproc.h \
pg_language.h pg_largeobject.h pg_aggregate.h pg_statistic.h \
pg_rewrite.h pg_trigger.h pg_listener.h pg_description.h \
- pg_database.h pg_shadow.h pg_group.h pg_log.h indexing.h \
+ pg_database.h pg_shadow.h pg_group.h indexing.h \
)
pg_includes := $(sort -I$(top_srcdir)/src/include -I$(top_builddir)/src/include)
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/catalog/heap.c,v 1.174 2001/08/10 18:57:33 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/catalog/heap.c,v 1.175 2001/08/25 18:52:41 tgl Exp $
*
*
* INTERFACE ROUTINES
tblNode = InvalidOid;
relid = RelOid_pg_database;
}
- else if (strcmp(LogRelationName, relname) == 0)
- {
- tblNode = InvalidOid;
- relid = RelOid_pg_log;
- }
else
{
relid = newoid();
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/parser/gram.y,v 2.247 2001/08/21 16:36:03 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/parser/gram.y,v 2.248 2001/08/25 18:52:41 tgl Exp $
*
* HISTORY
* AUTHOR DATE MAJOR EVENT
#include <ctype.h>
#include "access/htup.h"
-#include "catalog/catname.h"
#include "catalog/pg_type.h"
#include "nodes/params.h"
#include "nodes/parsenodes.h"
}
| ColId
{
- /* disallow refs to variable system tables */
- if (strcmp(LogRelationName, $1) == 0)
- elog(ERROR,"%s cannot be accessed by users",$1);
- else
- $$ = $1;
+ $$ = $1;
}
;
-$Header: /cvsroot/pgsql/src/backend/storage/buffer/README,v 1.1 2001/07/06 21:04:25 tgl Exp $
+$Header: /cvsroot/pgsql/src/backend/storage/buffer/README,v 1.2 2001/08/25 18:52:42 tgl Exp $
Notes about shared buffer access rules
--------------------------------------
is little or no risk of conflicting update; what's more, if there did
manage to be a conflict it would merely mean that one bit-update would
be lost and need to be done again later. These four bits are only hints
-(they cache the results of transaction status lookups in pg_log), so no
+(they cache the results of transaction status lookups in pg_clog), so no
great harm is done if they get reset to zero by conflicting updates.
5. To physically remove a tuple or compact free space on a page, one
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/storage/ipc/ipci.c,v 1.41 2001/06/27 23:31:39 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/ipc/ipci.c,v 1.42 2001/08/25 18:52:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include <sys/types.h>
#include "miscadmin.h"
+#include "access/clog.h"
#include "access/xlog.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
size = BufferShmemSize();
size += LockShmemSize(maxBackends);
size += XLOGShmemSize();
+ size += CLOGShmemSize();
size += SLockShmemSize();
size += SInvalShmemSize(maxBackends);
size += FreeSpaceShmemSize();
InitShmemAllocation(seghdr);
/*
- * Set up xlog and buffers
+ * Set up xlog, clog, and buffers
*/
XLOGShmemInit();
+ CLOGShmemInit();
InitBufferPool();
/*
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/storage/ipc/sinval.c,v 1.38 2001/08/23 23:06:37 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/ipc/sinval.c,v 1.39 2001/08/25 18:52:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
*--------------------
*/
- ReadNewTransactionId(&(snapshot->xmax));
+ snapshot->xmax = ReadNewTransactionId();
for (index = 0; index < segP->lastBackend; index++)
{
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/storage/lmgr/lmgr.c,v 1.49 2001/07/09 22:18:33 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/storage/lmgr/lmgr.c,v 1.50 2001/08/25 18:52:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
/*
* Transaction was committed/aborted/crashed - we have to update
- * pg_log if transaction is still marked as running.
+ * pg_clog if transaction is still marked as running.
*/
if (!TransactionIdDidCommit(xid) && !TransactionIdDidAbort(xid))
TransactionIdAbort(xid);
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/utils/cache/relcache.c,v 1.142 2001/08/10 18:57:37 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/utils/cache/relcache.c,v 1.143 2001/08/25 18:52:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include "catalog/indexing.h"
#include "catalog/pg_attrdef.h"
#include "catalog/pg_index.h"
-#include "catalog/pg_log.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_relcheck.h"
#include "catalog/pg_rewrite.h"
static FormData_pg_attribute Desc_pg_attribute[Natts_pg_attribute] = {Schema_pg_attribute};
static FormData_pg_attribute Desc_pg_proc[Natts_pg_proc] = {Schema_pg_proc};
static FormData_pg_attribute Desc_pg_type[Natts_pg_type] = {Schema_pg_type};
-static FormData_pg_attribute Desc_pg_log[Natts_pg_log] = {Schema_pg_log};
/*
* Hash tables that index the relation cache
formrdesc(AttributeRelationName, Natts_pg_attribute, Desc_pg_attribute);
formrdesc(ProcedureRelationName, Natts_pg_proc, Desc_pg_proc);
formrdesc(TypeRelationName, Natts_pg_type, Desc_pg_type);
- formrdesc(LogRelationName, Natts_pg_log, Desc_pg_log);
/*
* init_irels() used to be called here. It is changed to be called in
fixrdesc(AttributeRelationName);
fixrdesc(ProcedureRelationName);
fixrdesc(TypeRelationName);
-
- /*
- * We don't bother to update the entries for pg_log.
- */
}
}
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/utils/init/globals.c,v 1.58 2001/06/12 05:55:50 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/utils/init/globals.c,v 1.59 2001/08/25 18:52:42 tgl Exp $
*
* NOTES
* Globals used all over the place should be declared here and not
GroupRelationName,
GroupNameIndex,
GroupSysidIndex,
- LogRelationName,
ShadowRelationName,
ShadowNameIndex,
ShadowSysidIndex,
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/utils/init/postinit.c,v 1.87 2001/06/16 22:58:16 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/utils/init/postinit.c,v 1.88 2001/08/25 18:52:42 tgl Exp $
*
*
*-------------------------------------------------------------------------
elog(FATAL, "InitPostgres: bad backend id %d", MyBackendId);
/*
- * Initialize the transaction system and the relation descriptor
- * cache. Note we have to make certain the lock manager is off while
- * we do this.
+ * Initialize the transaction system override state.
*/
- AmiTransactionOverride(IsBootstrapProcessingMode());
- LockDisable(true);
+ AmiTransactionOverride(bootstrap);
/*
- * Part of the initialization processing done here sets a read lock on
- * pg_log. Since locking is disabled the set doesn't have intended
- * effect of locking out writers, but this is ok, since we only lock
- * it to examine AMI transaction status, and this is never written
- * after initdb is done. -mer 15 June 1992
+ * Initialize the relation descriptor cache.
+ * The pre-allocated reldescs are created here.
*/
- RelationCacheInitialize(); /* pre-allocated reldescs created here */
-
- InitializeTransactionSystem(); /* pg_log,etc init/crash recovery
- * here */
-
- LockDisable(false);
+ RelationCacheInitialize();
/*
* Initialize the access methods. Does not touch files (?) - thomas
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/utils/misc/Attic/database.c,v 1.47 2001/05/30 20:52:34 momjian Exp $
+ * $Header: /cvsroot/pgsql/src/backend/utils/misc/Attic/database.c,v 1.48 2001/08/25 18:52:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
* happens right here. We cannot really determine if the
* tuple is valid without checking transaction commit status,
* and the only way to do that at init time is to paw over
- * pg_log by hand, too. Instead of checking, we assume that
+ * pg_clog by hand, too. Instead of checking, we assume that
* the inserting transaction committed, and that any deleting
* transaction did also, unless shown otherwise by on-row
* commit status bits.
*
*
* IDENTIFICATION
- * $Header: /cvsroot/pgsql/src/backend/utils/time/tqual.c,v 1.40 2001/08/23 23:06:38 tgl Exp $
+ * $Header: /cvsroot/pgsql/src/backend/utils/time/tqual.c,v 1.41 2001/08/25 18:52:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
* If the inserting transaction aborted, then the tuple was never visible
* to any other transaction, so we can delete it immediately.
*
- * NOTE: must check TransactionIdIsInProgress (which looks in shared mem)
+ * NOTE: must check TransactionIdIsInProgress (which looks in PROC array)
* before TransactionIdDidCommit/TransactionIdDidAbort (which look in
- * pg_log). Otherwise we have a race condition where we might decide
+ * pg_clog). Otherwise we have a race condition where we might decide
* that a just-committed transaction crashed, because none of the tests
- * succeed. xact.c is careful to record commit/abort in pg_log before
- * it unsets MyProc->xid in shared memory.
+ * succeed. xact.c is careful to record commit/abort in pg_clog before
+ * it unsets MyProc->xid in PROC array.
*/
if (!(tuple->t_infomask & HEAP_XMIN_COMMITTED))
{
# Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
# Portions Copyright (c) 1994, Regents of the University of California
#
-# $Header: /cvsroot/pgsql/src/bin/initdb/Attic/initdb.sh,v 1.133 2001/08/25 00:31:17 petere Exp $
+# $Header: /cvsroot/pgsql/src/bin/initdb/Attic/initdb.sh,v 1.134 2001/08/25 18:52:42 tgl Exp $
#
#-------------------------------------------------------------------------
mkdir "$PGDATA"/pg_xlog || exit_nicely
echo "ok"
fi
+ if [ ! -d "$PGDATA"/pg_clog ]
+ then
+ $ECHO_N "creating directory $PGDATA/pg_clog... "$ECHO_C
+ mkdir "$PGDATA"/pg_clog || exit_nicely
+ echo "ok"
+ fi
fi
# pg_upgrade: update a database without needing a full dump/reload cycle.
# CAUTION: read the manual page before trying to use this!
-# $Header: /cvsroot/pgsql/src/bin/pg_dump/Attic/pg_upgrade,v 1.16 2000/12/18 16:30:07 momjian Exp $
+# $Header: /cvsroot/pgsql/src/bin/pg_dump/Attic/pg_upgrade,v 1.17 2001/08/25 18:52:42 tgl Exp $
#
# NOTE: we must be sure to update the version-checking code a few dozen lines
# below for each new PostgreSQL release.
echo "Input script $INPUT complete, fixing row commit statuses..."
# Now vacuum each result database to mark all system-table rows as committed,
-# because when pg_log is replaced with the saved version, the transaction
+# because when pg_clog is replaced with the saved version, the transaction
# statuses will no longer match the data. VACUUM will force the on-row
-# status flags to the right value so that pg_log will not matter anymore.
+# status flags to the right value so that pg_clog will not matter anymore.
# Note: we used to try to do this as part of the previous step, but that
# risks permissions problems if VACUUM is run as the wrong user.
# Note: the initial VACUUM does template1, then we do everything else.
fi
done
-mv -f $OLDDIR/pg_log data
+mv -f $OLDDIR/pg_clog data
mv -f $OLDDIR/pg_variable data
echo "You must stop/start the postmaster before doing anything else."
--- /dev/null
+/*
+ * clog.h
+ *
+ * PostgreSQL transaction-commit-log manager
+ *
+ * Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * $Id: clog.h,v 1.1 2001/08/25 18:52:42 tgl Exp $
+ */
+#ifndef CLOG_H
+#define CLOG_H
+
+#include "access/xlog.h"
+
+/*
+ * Possible transaction statuses --- note that all-zeroes is the initial
+ * state.
+ */
+typedef int XidStatus;
+
+#define TRANSACTION_STATUS_IN_PROGRESS 0x00
+#define TRANSACTION_STATUS_COMMITTED 0x01
+#define TRANSACTION_STATUS_ABORTED 0x02
+/* 0x03 is available without changing commit log space allocation */
+
+
+extern void TransactionIdSetStatus(TransactionId xid, XidStatus status);
+extern XidStatus TransactionIdGetStatus(TransactionId xid);
+
+extern int CLOGShmemSize(void);
+extern void CLOGShmemInit(void);
+extern void BootStrapCLOG(void);
+extern void StartupCLOG(void);
+extern void ShutdownCLOG(void);
+extern void CheckPointCLOG(void);
+extern void ExtendCLOG(TransactionId newestXact);
+extern void TruncateCLOG(TransactionId oldestXact);
+
+/* XLOG stuff */
+#define CLOG_ZEROPAGE 0x00
+
+extern void clog_redo(XLogRecPtr lsn, XLogRecord *record);
+extern void clog_undo(XLogRecPtr lsn, XLogRecord *record);
+extern void clog_desc(char *buf, uint8 xl_info, char *rec);
+
+#endif /* CLOG_H */
/*
- *
* rmgr.h
*
* Resource managers definition
*
+ * $Id: rmgr.h,v 1.6 2001/08/25 18:52:42 tgl Exp $
*/
#ifndef RMGR_H
#define RMGR_H
#define RM_XLOG_ID 0
#define RM_XACT_ID 1
#define RM_SMGR_ID 2
+#define RM_CLOG_ID 3
#define RM_HEAP_ID 10
#define RM_BTREE_ID 11
#define RM_HASH_ID 12
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Id: transam.h,v 1.38 2001/08/23 23:06:38 tgl Exp $
+ * $Id: transam.h,v 1.39 2001/08/25 18:52:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#ifndef TRANSAM_H
#define TRANSAM_H
-#include "storage/bufmgr.h"
+#include "storage/spin.h"
/* ----------------
* Special transaction ID values
*
- * We do not use any transaction IDs less than 512 --- this leaves the first
- * 128 bytes of pg_log available for special purposes such as version number
- * storage. (Currently, we do not actually use them for anything.)
- *
- * BootstrapTransactionId is the XID for "bootstrap" operations. It should
+ * BootstrapTransactionId is the XID for "bootstrap" operations, and
+ * FrozenTransactionId is used for very old tuples. Both should
* always be considered valid.
*
* FirstNormalTransactionId is the first "normal" transaction id.
* ----------------
*/
#define InvalidTransactionId ((TransactionId) 0)
-#define DisabledTransactionId ((TransactionId) 1)
-#define BootstrapTransactionId ((TransactionId) 512)
-#define FirstNormalTransactionId ((TransactionId) 514)
+#define BootstrapTransactionId ((TransactionId) 1)
+#define FrozenTransactionId ((TransactionId) 2)
+#define FirstNormalTransactionId ((TransactionId) 3)
/* ----------------
* transaction ID manipulation macros
(dest) = FirstNormalTransactionId; \
} while(0)
-/* ----------------
- * transaction status values
- *
- * someday we will use "11" = 3 = XID_COMMIT_CHILD to mean the
- * commiting of child xactions.
- * ----------------
- */
-#define XID_INPROGRESS 0 /* transaction in progress */
-#define XID_ABORT 1 /* transaction aborted */
-#define XID_COMMIT 2 /* transaction commited */
-#define XID_COMMIT_CHILD 3 /* child xact commited */
-
-typedef unsigned char XidStatus; /* (2 bits) */
/* ----------
* Object ID (OID) zero is InvalidOid.
/*
* prototypes for functions in transam/transam.c
*/
-extern void InitializeTransactionLog(void);
+extern void AmiTransactionOverride(bool flag);
extern bool TransactionIdDidCommit(TransactionId transactionId);
extern bool TransactionIdDidAbort(TransactionId transactionId);
extern void TransactionIdCommit(TransactionId transactionId);
extern void TransactionIdAbort(TransactionId transactionId);
-/* in transam/transsup.c */
-extern void AmiTransactionOverride(bool flag);
-extern void TransComputeBlockNumber(Relation relation,
- TransactionId transactionId, BlockNumber *blockNumberOutP);
-extern XidStatus TransBlockNumberGetXidStatus(Relation relation,
- BlockNumber blockNumber, TransactionId xid, bool *failP);
-extern void TransBlockNumberSetXidStatus(Relation relation,
- BlockNumber blockNumber, TransactionId xid, XidStatus xstatus,
- bool *failP);
-
/* in transam/varsup.c */
-extern void GetNewTransactionId(TransactionId *xid);
-extern void ReadNewTransactionId(TransactionId *xid);
+extern TransactionId GetNewTransactionId(void);
+extern TransactionId ReadNewTransactionId(void);
extern Oid GetNewObjectId(void);
extern void CheckMaxObjectId(Oid assigned_oid);
* ----------------
*/
-/* in transam.c */
-extern Relation LogRelation;
-
/* in xact.c */
extern bool AMI_OVERRIDE;
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Id: xact.h,v 1.34 2001/07/12 04:11:13 tgl Exp $
+ * $Id: xact.h,v 1.35 2001/08/25 18:52:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#define TRANS_INPROGRESS 2
#define TRANS_COMMIT 3
#define TRANS_ABORT 4
-#define TRANS_DISABLED 5
/* ----------------
* transaction block states
extern void SetTransactionFlushEnabled(bool state);
extern bool IsAbortedTransactionBlockState(void);
-extern void OverrideTransactionSystem(bool flag);
extern TransactionId GetCurrentTransactionId(void);
extern CommandId GetCurrentCommandId(void);
extern CommandId GetScanCommandId(void);
extern bool CommandIdIsCurrentCommandId(CommandId cid);
extern bool CommandIdGEScanCommandId(CommandId cid);
extern void CommandCounterIncrement(void);
-extern void InitializeTransactionSystem(void);
extern void StartTransactionCommand(void);
extern void CommitTransactionCommand(void);
extern void AbortCurrentTransaction(void);
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Id: xlog.h,v 1.24 2001/07/19 02:12:35 tgl Exp $
+ * $Id: xlog.h,v 1.25 2001/08/25 18:52:42 tgl Exp $
*/
#ifndef XLOG_H
#define XLOG_H
#include "access/rmgr.h"
#include "access/transam.h"
#include "access/xlogdefs.h"
+#include "storage/bufmgr.h"
#include "utils/pg_crc.h"
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Id: xlogdefs.h,v 1.3 2001/03/22 04:00:32 momjian Exp $
+ * $Id: xlogdefs.h,v 1.4 2001/08/25 18:52:42 tgl Exp $
*/
#ifndef XLOG_DEFS_H
#define XLOG_DEFS_H
/*
* StartUpID (SUI) - system startups counter. It's to allow removing
- * pg_log after shutdown, in future.
+ * pg_clog after shutdown, in future.
*/
typedef uint32 StartUpID;
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Id: xlogutils.h,v 1.7 2001/03/22 04:00:32 momjian Exp $
+ * $Id: xlogutils.h,v 1.8 2001/08/25 18:52:42 tgl Exp $
*/
#ifndef XLOG_UTILS_H
#define XLOG_UTILS_H
#include "access/rmgr.h"
+#include "storage/buf.h"
+#include "storage/itemptr.h"
#include "utils/rel.h"
extern int XLogIsOwnerOfTuple(RelFileNode hnode, ItemPointer iptr,
TransactionId xid, CommandId cid);
extern bool XLogIsValidTuple(RelFileNode hnode, ItemPointer iptr);
-extern void XLogOpenLogRelation(void);
-
extern void XLogInitRelationCache(void);
extern void XLogCloseRelationCache(void);
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Id: catname.h,v 1.19 2001/05/14 20:30:21 momjian Exp $
+ * $Id: catname.h,v 1.20 2001/08/25 18:52:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#define LanguageRelationName "pg_language"
#define LargeObjectRelationName "pg_largeobject"
#define ListenerRelationName "pg_listener"
-#define LogRelationName "pg_log"
#define OperatorClassRelationName "pg_opclass"
#define OperatorRelationName "pg_operator"
#define ProcedureRelationName "pg_proc"
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Id: catversion.h,v 1.91 2001/08/21 16:36:05 tgl Exp $
+ * $Id: catversion.h,v 1.92 2001/08/25 18:52:42 tgl Exp $
*
*-------------------------------------------------------------------------
*/
*/
/* yyyymmddN */
-#define CATALOG_VERSION_NO 200108211
+#define CATALOG_VERSION_NO 200108241
#endif
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Id: pg_attribute.h,v 1.74 2001/08/10 18:57:40 tgl Exp $
+ * $Id: pg_attribute.h,v 1.75 2001/08/25 18:52:42 tgl Exp $
*
* NOTES
* the genbki.sh script reads this file and generates .bki
DATA(insert ( 1259 tableoid 26 0 4 -7 0 -1 -1 t p f i f f));
/* ----------------
- * pg_log - this relation is modified by special purpose access
- * method code. The following is garbage but is needed
- * so that the reldesc code works properly.
+ * pg_xactlock - this is not a real relation, but is a placeholder
+ * to allow a relation OID to be used for transaction
+ * waits. We need a pg_xactlock entry in pg_class only to
+ * ensure that that OID can never be allocated to a real
+ * table; and this entry is just to link to that one.
* ----------------
*/
-#define Schema_pg_log \
-{ 1269, {"logfoo"}, 26, 0, 4, 1, 0, -1, -1, true, 'p', false, 'i', false, false }
-
-DATA(insert ( 1269 logfoo 26 0 4 1 0 -1 -1 t p f i f f));
-
-/* ----------------
- * pg_xactlock - this relation is modified by special purpose access
- * method code. The following is garbage but is needed
- * so that the reldesc code works properly.
- * ----------------
- */
-#define Schema_pg_xactlock \
-{ 376, {"xactlockfoo"}, 26, 0, 4, 1, 0, -1, -1, true, 'p', false, 'i', false, false }
-
DATA(insert ( 376 xactlockfoo 26 0 4 1 0 -1 -1 t p f i f f));
#endif /* PG_ATTRIBUTE_H */
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Id: pg_class.h,v 1.52 2001/08/10 18:57:40 tgl Exp $
+ * $Id: pg_class.h,v 1.53 2001/08/25 18:52:43 tgl Exp $
*
* NOTES
* the genbki.sh script reads this file and generates .bki
DESCR("");
DATA(insert OID = 1262 ( pg_database 88 PGUID 0 1262 0 0 0 0 f t r 7 0 0 0 0 0 t f f f _null_ ));
DESCR("");
-DATA(insert OID = 1269 ( pg_log 99 PGUID 0 1269 0 0 0 0 f t s 1 0 0 0 0 0 f f f f _null_ ));
-DESCR("");
DATA(insert OID = 376 ( pg_xactlock 0 PGUID 0 0 0 0 0 0 f t s 1 0 0 0 0 0 f f f f _null_ ));
DESCR("");
#define RelOid_pg_shadow 1260
#define RelOid_pg_group 1261
#define RelOid_pg_database 1262
-#define RelOid_pg_log 1269
/* Xact lock pseudo-table */
#define XactLockTableId 376
+++ /dev/null
-/*-------------------------------------------------------------------------
- *
- * pg_log.h
- * the system log relation "pg_log" is not a "heap" relation.
- * it is automatically created by the transam/ code and the
- * information here is all bogus and is just here to make the
- * relcache code happy.
- *
- *
- * Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
- * Portions Copyright (c) 1994, Regents of the University of California
- *
- * $Id: pg_log.h,v 1.9 2001/08/10 18:57:40 tgl Exp $
- *
- * NOTES
- * The structures and macros used by the transam/ code
- * to access pg_log should some day go here -cim 6/18/90
- *
- *-------------------------------------------------------------------------
- */
-#ifndef PG_LOG_H
-#define PG_LOG_H
-
-/* ----------------
- * postgres.h contains the system type definintions and the
- * CATALOG(), BOOTSTRAP and DATA() sugar words so this file
- * can be read by both genbki.sh and the C compiler.
- * ----------------
- */
-
-CATALOG(pg_log) BOOTSTRAP BKI_WITHOUT_OIDS
-{
- Oid logfoo;
-} FormData_pg_log;
-
-typedef FormData_pg_log *Form_pg_log;
-
-#define Natts_pg_log 1
-#define Anum_pg_log_logfoo 1
-
-#endif /* PG_LOG_H */
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Id: pg_type.h,v 1.109 2001/06/12 05:55:50 tgl Exp $
+ * $Id: pg_type.h,v 1.110 2001/08/25 18:52:43 tgl Exp $
*
* NOTES
* the genbki.sh script reads this file and generates .bki
DATA(insert OID = 86 ( pg_shadow PGUID 4 4 t c t \054 1260 0 int4in int4out int4in int4out i p _null_));
DATA(insert OID = 87 ( pg_group PGUID 4 4 t c t \054 1261 0 int4in int4out int4in int4out i p _null_));
DATA(insert OID = 88 ( pg_database PGUID 4 4 t c t \054 1262 0 int4in int4out int4in int4out i p _null_));
-DATA(insert OID = 99 ( pg_log PGUID 4 4 t c t \054 1269 0 int4in int4out int4in int4out i p _null_));
/* OIDS 100 - 199 */
* Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
- * $Id: bufpage.h,v 1.42 2001/05/14 22:06:41 momjian Exp $
+ * $Id: bufpage.h,v 1.43 2001/08/25 18:52:43 tgl Exp $
*
*-------------------------------------------------------------------------
*/
* obviously, a page is not formatted before it is initialized with by
* a call to PageInit.
*
- * The contents of the special pg_log tables are raw disk blocks with
- * special formats. these are the only "access methods" that need not
- * write disk pages.
- *
* NOTES:
*
* linp1..N form an ItemId array. ItemPointers point into this array