]> granicus.if.org Git - postgresql/commitdiff
Don't waste the last segment of each 4GB logical log file.
authorHeikki Linnakangas <heikki.linnakangas@iki.fi>
Sun, 24 Jun 2012 15:06:38 +0000 (18:06 +0300)
committerHeikki Linnakangas <heikki.linnakangas@iki.fi>
Sun, 24 Jun 2012 15:35:29 +0000 (18:35 +0300)
The comments claimed that wasting the last segment made it easier to do
calculations with XLogRecPtrs, because you don't have problems representing
last-byte-position-plus-1 that way. In my experience, however, it only made
things more complicated, because the there was two ways to represent the
boundary at the beginning of a logical log file: logid = n+1 and xrecoff = 0,
or as xlogid = n and xrecoff = 4GB - XLOG_SEG_SIZE. Some functions were
picky about which representation was used.

Also, use a 64-bit segment number instead of the log/seg combination, to
point to a certain WAL segment. We assume that all platforms have a working
64-bit integer type nowadays.

This is an incompatible change in WAL format, so bumping WAL version number.

12 files changed:
src/backend/access/transam/xlog.c
src/backend/access/transam/xlogfuncs.c
src/backend/postmaster/checkpointer.c
src/backend/replication/basebackup.c
src/backend/replication/walreceiver.c
src/backend/replication/walsender.c
src/bin/pg_basebackup/pg_receivexlog.c
src/bin/pg_basebackup/receivelog.c
src/bin/pg_resetxlog/pg_resetxlog.c
src/include/access/xlog.h
src/include/access/xlog_internal.h
src/include/access/xlogdefs.h

index 0d68760e81260373f4fc077895e4c986a26e5bc7..2f9209f3b8e7a1e29aeb9a293b7def49d5aad6df 100644 (file)
@@ -385,8 +385,7 @@ typedef struct XLogCtlData
        uint32          ckptXidEpoch;   /* nextXID & epoch of latest checkpoint */
        TransactionId ckptXid;
        XLogRecPtr      asyncXactLSN;   /* LSN of newest async commit/abort */
-       uint32          lastRemovedLog; /* latest removed/recycled XLOG segment */
-       uint32          lastRemovedSeg;
+       XLogSegNo       lastRemovedSegNo; /* latest removed/recycled XLOG segment */
 
        /* Protected by WALWriteLock: */
        XLogCtlWrite Write;
@@ -494,11 +493,13 @@ static ControlFileData *ControlFile = NULL;
 
 /* Construct XLogRecPtr value for current insertion point */
 #define INSERT_RECPTR(recptr,Insert,curridx)  \
-       ( \
-         (recptr).xlogid = XLogCtl->xlblocks[curridx].xlogid, \
-         (recptr).xrecoff = \
-               XLogCtl->xlblocks[curridx].xrecoff - INSERT_FREESPACE(Insert) \
-       )
+       do {                                                                                                                            \
+               (recptr).xlogid = XLogCtl->xlblocks[curridx].xlogid;                    \
+               (recptr).xrecoff =                                                                                              \
+                       XLogCtl->xlblocks[curridx].xrecoff - INSERT_FREESPACE(Insert); \
+               if (XLogCtl->xlblocks[curridx].xrecoff == 0)                                    \
+                       (recptr).xlogid = XLogCtl->xlblocks[curridx].xlogid - 1;        \
+       } while(0)
 
 #define PrevBufIdx(idx)                \
                (((idx) == 0) ? XLogCtl->XLogCacheBlck : ((idx) - 1))
@@ -524,12 +525,11 @@ static XLogwrtResult LogwrtResult = {{0, 0}, {0, 0}};
 /*
  * openLogFile is -1 or a kernel FD for an open log file segment.
  * When it's open, openLogOff is the current seek offset in the file.
- * openLogId/openLogSeg identify the segment.  These variables are only
+ * openLogSegNo identifies the segment.  These variables are only
  * used to write the XLOG, and so will normally refer to the active segment.
  */
 static int     openLogFile = -1;
-static uint32 openLogId = 0;
-static uint32 openLogSeg = 0;
+static XLogSegNo openLogSegNo = 0;
 static uint32 openLogOff = 0;
 
 /*
@@ -541,8 +541,7 @@ static uint32 openLogOff = 0;
  * the currently open file from.
  */
 static int     readFile = -1;
-static uint32 readId = 0;
-static uint32 readSeg = 0;
+static XLogSegNo readSegNo = 0;
 static uint32 readOff = 0;
 static uint32 readLen = 0;
 static int     readSource = 0;         /* XLOG_FROM_* code */
@@ -611,13 +610,12 @@ typedef struct xl_restore_point
 
 
 static void XLogArchiveNotify(const char *xlog);
-static void XLogArchiveNotifySeg(uint32 log, uint32 seg);
+static void XLogArchiveNotifySeg(XLogSegNo segno);
 static bool XLogArchiveCheckDone(const char *xlog);
 static bool XLogArchiveIsBusy(const char *xlog);
 static void XLogArchiveCleanup(const char *xlog);
 static void readRecoveryCommandFile(void);
-static void exitArchiveRecovery(TimeLineID endTLI,
-                                       uint32 endLogId, uint32 endLogSeg);
+static void exitArchiveRecovery(TimeLineID endTLI, XLogSegNo endLogSegNo);
 static bool recoveryStopsHere(XLogRecord *record, bool *includeThis);
 static void recoveryPausesHere(void);
 static void SetLatestXTime(TimestampTz xtime);
@@ -626,20 +624,19 @@ static void CheckRequiredParameterValues(void);
 static void XLogReportParameters(void);
 static void LocalSetXLogInsertAllowed(void);
 static void CheckPointGuts(XLogRecPtr checkPointRedo, int flags);
-static void KeepLogSeg(XLogRecPtr recptr, uint32 *logId, uint32 *logSeg);
+static void KeepLogSeg(XLogRecPtr recptr, XLogSegNo *logSegNo);
 
 static bool XLogCheckBuffer(XLogRecData *rdata, bool doPageWrites,
                                XLogRecPtr *lsn, BkpBlock *bkpb);
 static bool AdvanceXLInsertBuffer(bool new_segment);
-static bool XLogCheckpointNeeded(uint32 logid, uint32 logseg);
+static bool XLogCheckpointNeeded(XLogSegNo new_segno);
 static void XLogWrite(XLogwrtRqst WriteRqst, bool flexible, bool xlog_switch);
-static bool InstallXLogFileSegment(uint32 *log, uint32 *seg, char *tmppath,
+static bool InstallXLogFileSegment(XLogSegNo *segno, char *tmppath,
                                           bool find_free, int *max_advance,
                                           bool use_lock);
-static int XLogFileRead(uint32 log, uint32 seg, int emode, TimeLineID tli,
+static int XLogFileRead(XLogSegNo segno, int emode, TimeLineID tli,
                         int source, bool notexistOk);
-static int XLogFileReadAnyTLI(uint32 log, uint32 seg, int emode,
-                                  int sources);
+static int XLogFileReadAnyTLI(XLogSegNo segno, int emode, int sources);
 static bool XLogPageRead(XLogRecPtr *RecPtr, int emode, bool fetching_ckpt,
                         bool randAccess);
 static int     emode_for_corrupt_record(int emode, XLogRecPtr RecPtr);
@@ -649,7 +646,7 @@ static bool RestoreArchivedFile(char *path, const char *xlogfname,
 static void ExecuteRecoveryCommand(char *command, char *commandName,
                                           bool failOnerror);
 static void PreallocXlogFiles(XLogRecPtr endptr);
-static void RemoveOldXlogFiles(uint32 log, uint32 seg, XLogRecPtr endptr);
+static void RemoveOldXlogFiles(XLogSegNo segno, XLogRecPtr endptr);
 static void UpdateLastRemovedPtr(char *filename);
 static void ValidateXLOGDirectoryStructure(void);
 static void CleanupBackupHistory(void);
@@ -663,8 +660,7 @@ static bool existsTimeLineHistory(TimeLineID probeTLI);
 static bool rescanLatestTimeLine(void);
 static TimeLineID findNewestTimeLine(TimeLineID startTLI);
 static void writeTimeLineHistory(TimeLineID newTLI, TimeLineID parentTLI,
-                                        TimeLineID endTLI,
-                                        uint32 endLogId, uint32 endLogSeg);
+                                        TimeLineID endTLI, XLogSegNo endLogSegNo);
 static void WriteControlFile(void);
 static void ReadControlFile(void);
 static char *str_time(pg_time_t tnow);
@@ -996,12 +992,6 @@ begin:;
                LWLockRelease(WALInsertLock);
 
                RecPtr.xrecoff -= SizeOfXLogLongPHD;
-               if (RecPtr.xrecoff == 0)
-               {
-                       /* crossing a logid boundary */
-                       RecPtr.xlogid -= 1;
-                       RecPtr.xrecoff = XLogFileSize;
-               }
 
                LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
                LogwrtResult = XLogCtl->LogwrtResult;
@@ -1148,13 +1138,12 @@ begin:;
 
                /* Compute end address of old segment */
                OldSegEnd = XLogCtl->xlblocks[curridx];
-               OldSegEnd.xrecoff -= XLOG_BLCKSZ;
                if (OldSegEnd.xrecoff == 0)
                {
                        /* crossing a logid boundary */
                        OldSegEnd.xlogid -= 1;
-                       OldSegEnd.xrecoff = XLogFileSize;
                }
+               OldSegEnd.xrecoff -= XLOG_BLCKSZ;
 
                /* Make it look like we've written and synced all of old segment */
                LogwrtResult.Write = OldSegEnd;
@@ -1324,14 +1313,14 @@ XLogArchiveNotify(const char *xlog)
 }
 
 /*
- * Convenience routine to notify using log/seg representation of filename
+ * Convenience routine to notify using segment number representation of filename
  */
 static void
-XLogArchiveNotifySeg(uint32 log, uint32 seg)
+XLogArchiveNotifySeg(XLogSegNo segno)
 {
        char            xlog[MAXFNAMELEN];
 
-       XLogFileName(xlog, ThisTimeLineID, log, seg);
+       XLogFileName(xlog, ThisTimeLineID, segno);
        XLogArchiveNotify(xlog);
 }
 
@@ -1468,6 +1457,7 @@ AdvanceXLInsertBuffer(bool new_segment)
        XLogRecPtr      OldPageRqstPtr;
        XLogwrtRqst WriteRqst;
        XLogRecPtr      NewPageEndPtr;
+       XLogRecPtr      NewPageBeginPtr;
        XLogPageHeader NewPage;
 
        /*
@@ -1532,23 +1522,18 @@ AdvanceXLInsertBuffer(bool new_segment)
         * Now the next buffer slot is free and we can set it up to be the next
         * output page.
         */
-       NewPageEndPtr = XLogCtl->xlblocks[Insert->curridx];
+       NewPageBeginPtr = XLogCtl->xlblocks[Insert->curridx];
 
        if (new_segment)
        {
                /* force it to a segment start point */
-               NewPageEndPtr.xrecoff += XLogSegSize - 1;
-               NewPageEndPtr.xrecoff -= NewPageEndPtr.xrecoff % XLogSegSize;
+               if (NewPageBeginPtr.xrecoff % XLogSegSize != 0)
+                       XLByteAdvance(NewPageBeginPtr,
+                                                 XLogSegSize - NewPageBeginPtr.xrecoff % XLogSegSize);
        }
 
-       if (NewPageEndPtr.xrecoff >= XLogFileSize)
-       {
-               /* crossing a logid boundary */
-               NewPageEndPtr.xlogid += 1;
-               NewPageEndPtr.xrecoff = XLOG_BLCKSZ;
-       }
-       else
-               NewPageEndPtr.xrecoff += XLOG_BLCKSZ;
+       NewPageEndPtr = NewPageBeginPtr;
+       XLByteAdvance(NewPageEndPtr, XLOG_BLCKSZ);
        XLogCtl->xlblocks[nextidx] = NewPageEndPtr;
        NewPage = (XLogPageHeader) (XLogCtl->pages + nextidx * (Size) XLOG_BLCKSZ);
 
@@ -1570,8 +1555,7 @@ AdvanceXLInsertBuffer(bool new_segment)
 
        /* NewPage->xlp_info = 0; */    /* done by memset */
        NewPage   ->xlp_tli = ThisTimeLineID;
-       NewPage   ->xlp_pageaddr.xlogid = NewPageEndPtr.xlogid;
-       NewPage   ->xlp_pageaddr.xrecoff = NewPageEndPtr.xrecoff - XLOG_BLCKSZ;
+       NewPage   ->xlp_pageaddr = NewPageBeginPtr;
 
        /*
         * If online backup is not in progress, mark the header to indicate that
@@ -1609,33 +1593,20 @@ AdvanceXLInsertBuffer(bool new_segment)
 /*
  * Check whether we've consumed enough xlog space that a checkpoint is needed.
  *
- * logid/logseg indicate a log file that has just been filled up (or read
- * during recovery). We measure the distance from RedoRecPtr to logid/logseg
+ * new_segno indicates a log file that has just been filled up (or read
+ * during recovery). We measure the distance from RedoRecPtr to new_segno
  * and see if that exceeds CheckPointSegments.
  *
  * Note: it is caller's responsibility that RedoRecPtr is up-to-date.
  */
 static bool
-XLogCheckpointNeeded(uint32 logid, uint32 logseg)
+XLogCheckpointNeeded(XLogSegNo new_segno)
 {
-       /*
-        * A straight computation of segment number could overflow 32 bits. Rather
-        * than assuming we have working 64-bit arithmetic, we compare the
-        * highest-order bits separately, and force a checkpoint immediately when
-        * they change.
-        */
-       uint32          old_segno,
-                               new_segno;
-       uint32          old_highbits,
-                               new_highbits;
-
-       old_segno = (RedoRecPtr.xlogid % XLogSegSize) * XLogSegsPerFile +
-               (RedoRecPtr.xrecoff / XLogSegSize);
-       old_highbits = RedoRecPtr.xlogid / XLogSegSize;
-       new_segno = (logid % XLogSegSize) * XLogSegsPerFile + logseg;
-       new_highbits = logid / XLogSegSize;
-       if (new_highbits != old_highbits ||
-               new_segno >= old_segno + (uint32) (CheckPointSegments - 1))
+       XLogSegNo       old_segno;
+
+       XLByteToSeg(RedoRecPtr, old_segno);
+
+       if (new_segno >= old_segno + (uint64) (CheckPointSegments - 1))
                return true;
        return false;
 }
@@ -1716,7 +1687,7 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible, bool xlog_switch)
                LogwrtResult.Write = XLogCtl->xlblocks[curridx];
                ispartialpage = XLByteLT(WriteRqst.Write, LogwrtResult.Write);
 
-               if (!XLByteInPrevSeg(LogwrtResult.Write, openLogId, openLogSeg))
+               if (!XLByteInPrevSeg(LogwrtResult.Write, openLogSegNo))
                {
                        /*
                         * Switch to new logfile segment.  We cannot have any pending
@@ -1725,20 +1696,19 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible, bool xlog_switch)
                        Assert(npages == 0);
                        if (openLogFile >= 0)
                                XLogFileClose();
-                       XLByteToPrevSeg(LogwrtResult.Write, openLogId, openLogSeg);
+                       XLByteToPrevSeg(LogwrtResult.Write, openLogSegNo);
 
                        /* create/use new log file */
                        use_existent = true;
-                       openLogFile = XLogFileInit(openLogId, openLogSeg,
-                                                                          &use_existent, true);
+                       openLogFile = XLogFileInit(openLogSegNo, &use_existent, true);
                        openLogOff = 0;
                }
 
                /* Make sure we have the current logfile open */
                if (openLogFile < 0)
                {
-                       XLByteToPrevSeg(LogwrtResult.Write, openLogId, openLogSeg);
-                       openLogFile = XLogFileOpen(openLogId, openLogSeg);
+                       XLByteToPrevSeg(LogwrtResult.Write, openLogSegNo);
+                       openLogFile = XLogFileOpen(openLogSegNo);
                        openLogOff = 0;
                }
 
@@ -1775,9 +1745,9 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible, bool xlog_switch)
                                if (lseek(openLogFile, (off_t) startoffset, SEEK_SET) < 0)
                                        ereport(PANIC,
                                                        (errcode_for_file_access(),
-                                                        errmsg("could not seek in log file %u, "
-                                                                       "segment %u to offset %u: %m",
-                                                                       openLogId, openLogSeg, startoffset)));
+                                                        errmsg("could not seek in log file %s to offset %u: %m",
+                                                                       XLogFileNameP(ThisTimeLineID, openLogSegNo),
+                                                                       startoffset)));
                                openLogOff = startoffset;
                        }
 
@@ -1792,9 +1762,9 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible, bool xlog_switch)
                                        errno = ENOSPC;
                                ereport(PANIC,
                                                (errcode_for_file_access(),
-                                                errmsg("could not write to log file %u, segment %u "
+                                                errmsg("could not write to log file %s "
                                                                "at offset %u, length %lu: %m",
-                                                               openLogId, openLogSeg,
+                                                               XLogFileNameP(ThisTimeLineID, openLogSegNo),
                                                                openLogOff, (unsigned long) nbytes)));
                        }
 
@@ -1821,11 +1791,11 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible, bool xlog_switch)
                         */
                        if (finishing_seg || (xlog_switch && last_iteration))
                        {
-                               issue_xlog_fsync(openLogFile, openLogId, openLogSeg);
+                               issue_xlog_fsync(openLogFile, openLogSegNo);
                                LogwrtResult.Flush = LogwrtResult.Write;                /* end of page */
 
                                if (XLogArchivingActive())
-                                       XLogArchiveNotifySeg(openLogId, openLogSeg);
+                                       XLogArchiveNotifySeg(openLogSegNo);
 
                                Write->lastSegSwitchTime = (pg_time_t) time(NULL);
 
@@ -1836,11 +1806,10 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible, bool xlog_switch)
                                 * like a checkpoint is needed, forcibly update RedoRecPtr and
                                 * recheck.
                                 */
-                               if (IsUnderPostmaster &&
-                                       XLogCheckpointNeeded(openLogId, openLogSeg))
+                               if (IsUnderPostmaster && XLogCheckpointNeeded(openLogSegNo))
                                {
                                        (void) GetRedoRecPtr();
-                                       if (XLogCheckpointNeeded(openLogId, openLogSeg))
+                                       if (XLogCheckpointNeeded(openLogSegNo))
                                                RequestCheckpoint(CHECKPOINT_CAUSE_XLOG);
                                }
                        }
@@ -1877,15 +1846,15 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible, bool xlog_switch)
                        sync_method != SYNC_METHOD_OPEN_DSYNC)
                {
                        if (openLogFile >= 0 &&
-                               !XLByteInPrevSeg(LogwrtResult.Write, openLogId, openLogSeg))
+                               !XLByteInPrevSeg(LogwrtResult.Write, openLogSegNo))
                                XLogFileClose();
                        if (openLogFile < 0)
                        {
-                               XLByteToPrevSeg(LogwrtResult.Write, openLogId, openLogSeg);
-                               openLogFile = XLogFileOpen(openLogId, openLogSeg);
+                               XLByteToPrevSeg(LogwrtResult.Write, openLogSegNo);
+                               openLogFile = XLogFileOpen(openLogSegNo);
                                openLogOff = 0;
                        }
-                       issue_xlog_fsync(openLogFile, openLogId, openLogSeg);
+                       issue_xlog_fsync(openLogFile, openLogSegNo);
                }
                LogwrtResult.Flush = LogwrtResult.Write;
        }
@@ -2129,6 +2098,8 @@ XLogFlush(XLogRecPtr record)
                                else
                                {
                                        WriteRqstPtr = XLogCtl->xlblocks[Insert->curridx];
+                                       if (WriteRqstPtr.xrecoff == 0)
+                                               WriteRqstPtr.xlogid--;
                                        WriteRqstPtr.xrecoff -= freespace;
                                }
                                LWLockRelease(WALInsertLock);
@@ -2240,7 +2211,7 @@ XLogBackgroundFlush(void)
        {
                if (openLogFile >= 0)
                {
-                       if (!XLByteInPrevSeg(LogwrtResult.Write, openLogId, openLogSeg))
+                       if (!XLByteInPrevSeg(LogwrtResult.Write, openLogSegNo))
                        {
                                XLogFileClose();
                        }
@@ -2372,19 +2343,17 @@ XLogNeedsFlush(XLogRecPtr record)
  * in a critical section.
  */
 int
-XLogFileInit(uint32 log, uint32 seg,
-                        bool *use_existent, bool use_lock)
+XLogFileInit(XLogSegNo logsegno, bool *use_existent, bool use_lock)
 {
        char            path[MAXPGPATH];
        char            tmppath[MAXPGPATH];
        char       *zbuffer;
-       uint32          installed_log;
-       uint32          installed_seg;
+       XLogSegNo       installed_segno;
        int                     max_advance;
        int                     fd;
        int                     nbytes;
 
-       XLogFilePath(path, ThisTimeLineID, log, seg);
+       XLogFilePath(path, ThisTimeLineID, logsegno);
 
        /*
         * Try to use existent file (checkpoint maker may have created it already)
@@ -2398,8 +2367,7 @@ XLogFileInit(uint32 log, uint32 seg,
                        if (errno != ENOENT)
                                ereport(ERROR,
                                                (errcode_for_file_access(),
-                                                errmsg("could not open file \"%s\" (log file %u, segment %u): %m",
-                                                               path, log, seg)));
+                                                errmsg("could not open file \"%s\": %m", path)));
                }
                else
                        return fd;
@@ -2478,10 +2446,9 @@ XLogFileInit(uint32 log, uint32 seg,
         * has created the file while we were filling ours: if so, use ours to
         * pre-create a future log segment.
         */
-       installed_log = log;
-       installed_seg = seg;
+       installed_segno = logsegno;
        max_advance = XLOGfileslop;
-       if (!InstallXLogFileSegment(&installed_log, &installed_seg, tmppath,
+       if (!InstallXLogFileSegment(&installed_segno, tmppath,
                                                                *use_existent, &max_advance,
                                                                use_lock))
        {
@@ -2502,8 +2469,7 @@ XLogFileInit(uint32 log, uint32 seg,
        if (fd < 0)
                ereport(ERROR,
                                (errcode_for_file_access(),
-                  errmsg("could not open file \"%s\" (log file %u, segment %u): %m",
-                                 path, log, seg)));
+                  errmsg("could not open file \"%s\": %m", path)));
 
        elog(DEBUG2, "done creating and filling new WAL file");
 
@@ -2523,8 +2489,7 @@ XLogFileInit(uint32 log, uint32 seg,
  * emplacing a bogus file.
  */
 static void
-XLogFileCopy(uint32 log, uint32 seg,
-                        TimeLineID srcTLI, uint32 srclog, uint32 srcseg)
+XLogFileCopy(XLogSegNo destsegno, TimeLineID srcTLI, XLogSegNo srcsegno)
 {
        char            path[MAXPGPATH];
        char            tmppath[MAXPGPATH];
@@ -2536,7 +2501,7 @@ XLogFileCopy(uint32 log, uint32 seg,
        /*
         * Open the source file
         */
-       XLogFilePath(path, srcTLI, srclog, srcseg);
+       XLogFilePath(path, srcTLI, srcsegno);
        srcfd = BasicOpenFile(path, O_RDONLY | PG_BINARY, 0);
        if (srcfd < 0)
                ereport(ERROR,
@@ -2607,7 +2572,7 @@ XLogFileCopy(uint32 log, uint32 seg,
        /*
         * Now move the segment into place with its final name.
         */
-       if (!InstallXLogFileSegment(&log, &seg, tmppath, false, NULL, false))
+       if (!InstallXLogFileSegment(&destsegno, tmppath, false, NULL, false))
                elog(ERROR, "InstallXLogFileSegment should not have failed");
 }
 
@@ -2641,14 +2606,14 @@ XLogFileCopy(uint32 log, uint32 seg,
  * file into place.
  */
 static bool
-InstallXLogFileSegment(uint32 *log, uint32 *seg, char *tmppath,
+InstallXLogFileSegment(XLogSegNo *segno, char *tmppath,
                                           bool find_free, int *max_advance,
                                           bool use_lock)
 {
        char            path[MAXPGPATH];
        struct stat stat_buf;
 
-       XLogFilePath(path, ThisTimeLineID, *log, *seg);
+       XLogFilePath(path, ThisTimeLineID, *segno);
 
        /*
         * We want to be sure that only one process does this at a time.
@@ -2673,9 +2638,9 @@ InstallXLogFileSegment(uint32 *log, uint32 *seg, char *tmppath,
                                        LWLockRelease(ControlFileLock);
                                return false;
                        }
-                       NextLogSeg(*log, *seg);
+                       (*segno)++;
                        (*max_advance)--;
-                       XLogFilePath(path, ThisTimeLineID, *log, *seg);
+                       XLogFilePath(path, ThisTimeLineID, *segno);
                }
        }
 
@@ -2691,8 +2656,8 @@ InstallXLogFileSegment(uint32 *log, uint32 *seg, char *tmppath,
                        LWLockRelease(ControlFileLock);
                ereport(LOG,
                                (errcode_for_file_access(),
-                                errmsg("could not link file \"%s\" to \"%s\" (initialization of log file %u, segment %u): %m",
-                                               tmppath, path, *log, *seg)));
+                                errmsg("could not link file \"%s\" to \"%s\" (initialization of log file): %m",
+                                               tmppath, path)));
                return false;
        }
        unlink(tmppath);
@@ -2703,8 +2668,8 @@ InstallXLogFileSegment(uint32 *log, uint32 *seg, char *tmppath,
                        LWLockRelease(ControlFileLock);
                ereport(LOG,
                                (errcode_for_file_access(),
-                                errmsg("could not rename file \"%s\" to \"%s\" (initialization of log file %u, segment %u): %m",
-                                               tmppath, path, *log, *seg)));
+                                errmsg("could not rename file \"%s\" to \"%s\" (initialization of log file): %m",
+                                               tmppath, path)));
                return false;
        }
 #endif
@@ -2719,20 +2684,19 @@ InstallXLogFileSegment(uint32 *log, uint32 *seg, char *tmppath,
  * Open a pre-existing logfile segment for writing.
  */
 int
-XLogFileOpen(uint32 log, uint32 seg)
+XLogFileOpen(XLogSegNo segno)
 {
        char            path[MAXPGPATH];
        int                     fd;
 
-       XLogFilePath(path, ThisTimeLineID, log, seg);
+       XLogFilePath(path, ThisTimeLineID, segno);
 
        fd = BasicOpenFile(path, O_RDWR | PG_BINARY | get_sync_bit(sync_method),
                                           S_IRUSR | S_IWUSR);
        if (fd < 0)
                ereport(PANIC,
                                (errcode_for_file_access(),
-                  errmsg("could not open file \"%s\" (log file %u, segment %u): %m",
-                                 path, log, seg)));
+                                errmsg("could not open xlog file \"%s\": %m", path)));
 
        return fd;
 }
@@ -2744,7 +2708,7 @@ XLogFileOpen(uint32 log, uint32 seg)
  * Otherwise, it's assumed to be already available in pg_xlog.
  */
 static int
-XLogFileRead(uint32 log, uint32 seg, int emode, TimeLineID tli,
+XLogFileRead(XLogSegNo segno, int emode, TimeLineID tli,
                         int source, bool notfoundOk)
 {
        char            xlogfname[MAXFNAMELEN];
@@ -2752,7 +2716,7 @@ XLogFileRead(uint32 log, uint32 seg, int emode, TimeLineID tli,
        char            path[MAXPGPATH];
        int                     fd;
 
-       XLogFileName(xlogfname, tli, log, seg);
+       XLogFileName(xlogfname, tli, segno);
 
        switch (source)
        {
@@ -2771,7 +2735,7 @@ XLogFileRead(uint32 log, uint32 seg, int emode, TimeLineID tli,
 
                case XLOG_FROM_PG_XLOG:
                case XLOG_FROM_STREAM:
-                       XLogFilePath(path, tli, log, seg);
+                       XLogFilePath(path, tli, segno);
                        restoredFromArchive = false;
                        break;
 
@@ -2792,7 +2756,7 @@ XLogFileRead(uint32 log, uint32 seg, int emode, TimeLineID tli,
                bool            reload = false;
                struct stat statbuf;
 
-               XLogFilePath(xlogfpath, tli, log, seg);
+               XLogFilePath(xlogfpath, tli, segno);
                if (stat(xlogfpath, &statbuf) == 0)
                {
                        if (unlink(xlogfpath) != 0)
@@ -2821,8 +2785,7 @@ XLogFileRead(uint32 log, uint32 seg, int emode, TimeLineID tli,
                 * shmem. It's used as current standby flush position, and cascading
                 * walsenders try to send WAL records up to this location.
                 */
-               endptr.xlogid = log;
-               endptr.xrecoff = seg * XLogSegSize;
+               XLogSegNoOffsetToRecPtr(segno, 0, endptr);
                XLByteAdvance(endptr, XLogSegSize);
 
                SpinLockAcquire(&xlogctl->info_lck);
@@ -2857,8 +2820,7 @@ XLogFileRead(uint32 log, uint32 seg, int emode, TimeLineID tli,
        if (errno != ENOENT || !notfoundOk) /* unexpected failure? */
                ereport(PANIC,
                                (errcode_for_file_access(),
-                  errmsg("could not open file \"%s\" (log file %u, segment %u): %m",
-                                 path, log, seg)));
+                                errmsg("could not open file \"%s\": %m", path)));
        return -1;
 }
 
@@ -2868,7 +2830,7 @@ XLogFileRead(uint32 log, uint32 seg, int emode, TimeLineID tli,
  * This version searches for the segment with any TLI listed in expectedTLIs.
  */
 static int
-XLogFileReadAnyTLI(uint32 log, uint32 seg, int emode, int sources)
+XLogFileReadAnyTLI(XLogSegNo segno, int emode, int sources)
 {
        char            path[MAXPGPATH];
        ListCell   *cell;
@@ -2893,7 +2855,7 @@ XLogFileReadAnyTLI(uint32 log, uint32 seg, int emode, int sources)
 
                if (sources & XLOG_FROM_ARCHIVE)
                {
-                       fd = XLogFileRead(log, seg, emode, tli, XLOG_FROM_ARCHIVE, true);
+                       fd = XLogFileRead(segno, emode, tli, XLOG_FROM_ARCHIVE, true);
                        if (fd != -1)
                        {
                                elog(DEBUG1, "got WAL segment from archive");
@@ -2903,19 +2865,18 @@ XLogFileReadAnyTLI(uint32 log, uint32 seg, int emode, int sources)
 
                if (sources & XLOG_FROM_PG_XLOG)
                {
-                       fd = XLogFileRead(log, seg, emode, tli, XLOG_FROM_PG_XLOG, true);
+                       fd = XLogFileRead(segno, emode, tli, XLOG_FROM_PG_XLOG, true);
                        if (fd != -1)
                                return fd;
                }
        }
 
        /* Couldn't find it.  For simplicity, complain about front timeline */
-       XLogFilePath(path, recoveryTargetTLI, log, seg);
+       XLogFilePath(path, recoveryTargetTLI, segno);
        errno = ENOENT;
        ereport(emode,
                        (errcode_for_file_access(),
-                  errmsg("could not open file \"%s\" (log file %u, segment %u): %m",
-                                 path, log, seg)));
+                        errmsg("could not open file \"%s\": %m", path)));
        return -1;
 }
 
@@ -2941,8 +2902,8 @@ XLogFileClose(void)
        if (close(openLogFile))
                ereport(PANIC,
                                (errcode_for_file_access(),
-                                errmsg("could not close log file %u, segment %u: %m",
-                                               openLogId, openLogSeg)));
+                                errmsg("could not close log file %s: %m",
+                                               XLogFileNameP(ThisTimeLineID, openLogSegNo))));
        openLogFile = -1;
 }
 
@@ -2973,8 +2934,7 @@ RestoreArchivedFile(char *path, const char *xlogfname,
        int                     rc;
        bool            signaled;
        struct stat stat_buf;
-       uint32          restartLog;
-       uint32          restartSeg;
+       XLogSegNo       restartSegNo;
 
        /* In standby mode, restore_command might not be supplied */
        if (recoveryRestoreCommand == NULL)
@@ -3043,16 +3003,15 @@ RestoreArchivedFile(char *path, const char *xlogfname,
         */
        if (InRedo)
        {
-               XLByteToSeg(ControlFile->checkPointCopy.redo,
-                                       restartLog, restartSeg);
+               XLByteToSeg(ControlFile->checkPointCopy.redo, restartSegNo);
                XLogFileName(lastRestartPointFname,
                                         ControlFile->checkPointCopy.ThisTimeLineID,
-                                        restartLog, restartSeg);
+                                        restartSegNo);
                /* we shouldn't need anything earlier than last restart point */
                Assert(strcmp(lastRestartPointFname, xlogfname) <= 0);
        }
        else
-               XLogFileName(lastRestartPointFname, 0, 0, 0);
+               XLogFileName(lastRestartPointFname, 0, 0L);
 
        /*
         * construct the command to be executed
@@ -3247,8 +3206,7 @@ ExecuteRecoveryCommand(char *command, char *commandName, bool failOnSignal)
        const char *sp;
        int                     rc;
        bool            signaled;
-       uint32          restartLog;
-       uint32          restartSeg;
+       XLogSegNo       restartSegNo;
 
        Assert(command && commandName);
 
@@ -3258,11 +3216,10 @@ ExecuteRecoveryCommand(char *command, char *commandName, bool failOnSignal)
         * archive, though there is no requirement to do so.
         */
        LWLockAcquire(ControlFileLock, LW_SHARED);
-       XLByteToSeg(ControlFile->checkPointCopy.redo,
-                               restartLog, restartSeg);
+       XLByteToSeg(ControlFile->checkPointCopy.redo, restartSegNo);
        XLogFileName(lastRestartPointFname,
                                 ControlFile->checkPointCopy.ThisTimeLineID,
-                                restartLog, restartSeg);
+                                restartSegNo);
        LWLockRelease(ControlFileLock);
 
        /*
@@ -3343,18 +3300,17 @@ ExecuteRecoveryCommand(char *command, char *commandName, bool failOnSignal)
 static void
 PreallocXlogFiles(XLogRecPtr endptr)
 {
-       uint32          _logId;
-       uint32          _logSeg;
+       XLogSegNo       _logSegNo;
        int                     lf;
        bool            use_existent;
 
-       XLByteToPrevSeg(endptr, _logId, _logSeg);
+       XLByteToPrevSeg(endptr, _logSegNo);
        if ((endptr.xrecoff - 1) % XLogSegSize >=
                (uint32) (0.75 * XLogSegSize))
        {
-               NextLogSeg(_logId, _logSeg);
+               _logSegNo++;
                use_existent = true;
-               lf = XLogFileInit(_logId, _logSeg, &use_existent, true);
+               lf = XLogFileInit(_logSegNo, &use_existent, true);
                close(lf);
                if (!use_existent)
                        CheckpointStats.ckpt_segs_added++;
@@ -3366,14 +3322,13 @@ PreallocXlogFiles(XLogRecPtr endptr)
  * Returns 0/0 if no WAL segments have been removed since startup.
  */
 void
-XLogGetLastRemoved(uint32 *log, uint32 *seg)
+XLogGetLastRemoved(XLogSegNo *segno)
 {
        /* use volatile pointer to prevent code rearrangement */
        volatile XLogCtlData *xlogctl = XLogCtl;
 
        SpinLockAcquire(&xlogctl->info_lck);
-       *log = xlogctl->lastRemovedLog;
-       *seg = xlogctl->lastRemovedSeg;
+       *segno = xlogctl->lastRemovedSegNo;
        SpinLockRelease(&xlogctl->info_lck);
 }
 
@@ -3386,19 +3341,14 @@ UpdateLastRemovedPtr(char *filename)
 {
        /* use volatile pointer to prevent code rearrangement */
        volatile XLogCtlData *xlogctl = XLogCtl;
-       uint32          tli,
-                               log,
-                               seg;
+       uint32          tli;
+       XLogSegNo       segno;
 
-       XLogFromFileName(filename, &tli, &log, &seg);
+       XLogFromFileName(filename, &tli, &segno);
 
        SpinLockAcquire(&xlogctl->info_lck);
-       if (log > xlogctl->lastRemovedLog ||
-               (log == xlogctl->lastRemovedLog && seg > xlogctl->lastRemovedSeg))
-       {
-               xlogctl->lastRemovedLog = log;
-               xlogctl->lastRemovedSeg = seg;
-       }
+       if (segno > xlogctl->lastRemovedSegNo)
+               xlogctl->lastRemovedSegNo = segno;
        SpinLockRelease(&xlogctl->info_lck);
 }
 
@@ -3409,10 +3359,9 @@ UpdateLastRemovedPtr(char *filename)
  * whether we want to recycle rather than delete no-longer-wanted log files.
  */
 static void
-RemoveOldXlogFiles(uint32 log, uint32 seg, XLogRecPtr endptr)
+RemoveOldXlogFiles(XLogSegNo segno, XLogRecPtr endptr)
 {
-       uint32          endlogId;
-       uint32          endlogSeg;
+       XLogSegNo       endlogSegNo;
        int                     max_advance;
        DIR                *xldir;
        struct dirent *xlde;
@@ -3428,7 +3377,7 @@ RemoveOldXlogFiles(uint32 log, uint32 seg, XLogRecPtr endptr)
         * Initialize info about where to try to recycle to.  We allow recycling
         * segments up to XLOGfileslop segments beyond the current XLOG location.
         */
-       XLByteToPrevSeg(endptr, endlogId, endlogSeg);
+       XLByteToPrevSeg(endptr, endlogSegNo);
        max_advance = XLOGfileslop;
 
        xldir = AllocateDir(XLOGDIR);
@@ -3438,7 +3387,7 @@ RemoveOldXlogFiles(uint32 log, uint32 seg, XLogRecPtr endptr)
                                 errmsg("could not open transaction log directory \"%s\": %m",
                                                XLOGDIR)));
 
-       XLogFileName(lastoff, ThisTimeLineID, log, seg);
+       XLogFileName(lastoff, ThisTimeLineID, segno);
 
        elog(DEBUG2, "attempting to remove WAL segments older than log file %s",
                 lastoff);
@@ -3474,7 +3423,7 @@ RemoveOldXlogFiles(uint32 log, uint32 seg, XLogRecPtr endptr)
                                 * separate archive directory.
                                 */
                                if (lstat(path, &statbuf) == 0 && S_ISREG(statbuf.st_mode) &&
-                                       InstallXLogFileSegment(&endlogId, &endlogSeg, path,
+                                       InstallXLogFileSegment(&endlogSegNo, path,
                                                                                   true, &max_advance, true))
                                {
                                        ereport(DEBUG2,
@@ -3484,7 +3433,7 @@ RemoveOldXlogFiles(uint32 log, uint32 seg, XLogRecPtr endptr)
                                        /* Needn't recheck that slot on future iterations */
                                        if (max_advance > 0)
                                        {
-                                               NextLogSeg(endlogId, endlogSeg);
+                                               endlogSegNo++;
                                                max_advance--;
                                        }
                                }
@@ -3823,13 +3772,6 @@ ReadRecord(XLogRecPtr *RecPtr, int emode, bool fetching_ckpt)
                if (XLOG_BLCKSZ - (RecPtr->xrecoff % XLOG_BLCKSZ) < SizeOfXLogRecord)
                        NextLogPage(*RecPtr);
 
-               /* Check for crossing of xlog segment boundary */
-               if (RecPtr->xrecoff >= XLogFileSize)
-               {
-                       (RecPtr->xlogid)++;
-                       RecPtr->xrecoff = 0;
-               }
-
                /*
                 * If at page start, we must skip over the page header.  But we can't
                 * do that until we've read in the page, since the header size is
@@ -4013,12 +3955,7 @@ retry:
                for (;;)
                {
                        /* Calculate pointer to beginning of next page */
-                       pagelsn.xrecoff += XLOG_BLCKSZ;
-                       if (pagelsn.xrecoff >= XLogFileSize)
-                       {
-                               (pagelsn.xlogid)++;
-                               pagelsn.xrecoff = 0;
-                       }
+                       XLByteAdvance(pagelsn, XLOG_BLCKSZ);
                        /* Wait for the next page to become available */
                        if (!XLogPageRead(&pagelsn, emode, false, false))
                                return NULL;
@@ -4027,8 +3964,9 @@ retry:
                        if (!(((XLogPageHeader) readBuf)->xlp_info & XLP_FIRST_IS_CONTRECORD))
                        {
                                ereport(emode_for_corrupt_record(emode, *RecPtr),
-                                               (errmsg("there is no contrecord flag in log file %u, segment %u, offset %u",
-                                                               readId, readSeg, readOff)));
+                                               (errmsg("there is no contrecord flag in log segment %s, offset %u",
+                                                               XLogFileNameP(curFileTLI, readSegNo),
+                                                               readOff)));
                                goto next_record_is_invalid;
                        }
                        pageHeaderSize = XLogPageHeaderSize((XLogPageHeader) readBuf);
@@ -4036,10 +3974,13 @@ retry:
                        if (contrecord->xl_rem_len == 0 ||
                                total_len != (contrecord->xl_rem_len + gotlen))
                        {
+                               char fname[MAXFNAMELEN];
+                               XLogFileName(fname, curFileTLI, readSegNo);
                                ereport(emode_for_corrupt_record(emode, *RecPtr),
-                                               (errmsg("invalid contrecord length %u in log file %u, segment %u, offset %u",
+                                               (errmsg("invalid contrecord length %u in log segment %s, offset %u",
                                                                contrecord->xl_rem_len,
-                                                               readId, readSeg, readOff)));
+                                                               XLogFileNameP(curFileTLI, readSegNo),
+                                                               readOff)));
                                goto next_record_is_invalid;
                        }
                        len = XLOG_BLCKSZ - pageHeaderSize - SizeOfXLogContRecord;
@@ -4057,11 +3998,11 @@ retry:
                if (!RecordIsValid(record, *RecPtr, emode))
                        goto next_record_is_invalid;
                pageHeaderSize = XLogPageHeaderSize((XLogPageHeader) readBuf);
-               EndRecPtr.xlogid = readId;
-               EndRecPtr.xrecoff = readSeg * XLogSegSize + readOff +
-                       pageHeaderSize +
-                       MAXALIGN(SizeOfXLogContRecord + contrecord->xl_rem_len);
-
+               XLogSegNoOffsetToRecPtr(
+                       readSegNo,
+                       readOff + pageHeaderSize +
+                               MAXALIGN(SizeOfXLogContRecord + contrecord->xl_rem_len),
+                       EndRecPtr);
                ReadRecPtr = *RecPtr;
                /* needn't worry about XLOG SWITCH, it can't cross page boundaries */
                return record;
@@ -4121,21 +4062,24 @@ ValidXLOGHeader(XLogPageHeader hdr, int emode)
 {
        XLogRecPtr      recaddr;
 
-       recaddr.xlogid = readId;
-       recaddr.xrecoff = readSeg * XLogSegSize + readOff;
+       XLogSegNoOffsetToRecPtr(readSegNo, readOff, recaddr);
 
        if (hdr->xlp_magic != XLOG_PAGE_MAGIC)
        {
                ereport(emode_for_corrupt_record(emode, recaddr),
-                               (errmsg("invalid magic number %04X in log file %u, segment %u, offset %u",
-                                               hdr->xlp_magic, readId, readSeg, readOff)));
+                               (errmsg("invalid magic number %04X in log segment %s, offset %u",
+                                               hdr->xlp_magic,
+                                               XLogFileNameP(curFileTLI, readSegNo),
+                                               readOff)));
                return false;
        }
        if ((hdr->xlp_info & ~XLP_ALL_FLAGS) != 0)
        {
                ereport(emode_for_corrupt_record(emode, recaddr),
-                               (errmsg("invalid info bits %04X in log file %u, segment %u, offset %u",
-                                               hdr->xlp_info, readId, readSeg, readOff)));
+                               (errmsg("invalid info bits %04X in log segment %s, offset %u",
+                                               hdr->xlp_info,
+                                               XLogFileNameP(curFileTLI, readSegNo),
+                                               readOff)));
                return false;
        }
        if (hdr->xlp_info & XLP_LONG_HEADER)
@@ -4180,17 +4124,20 @@ ValidXLOGHeader(XLogPageHeader hdr, int emode)
        {
                /* hmm, first page of file doesn't have a long header? */
                ereport(emode_for_corrupt_record(emode, recaddr),
-                               (errmsg("invalid info bits %04X in log file %u, segment %u, offset %u",
-                                               hdr->xlp_info, readId, readSeg, readOff)));
+                               (errmsg("invalid info bits %04X in log segment %s, offset %u",
+                                               hdr->xlp_info,
+                                               XLogFileNameP(curFileTLI, readSegNo),
+                                               readOff)));
                return false;
        }
 
        if (!XLByteEQ(hdr->xlp_pageaddr, recaddr))
        {
                ereport(emode_for_corrupt_record(emode, recaddr),
-                               (errmsg("unexpected pageaddr %X/%X in log file %u, segment %u, offset %u",
+                               (errmsg("unexpected pageaddr %X/%X in log segment %s, offset %u",
                                                hdr->xlp_pageaddr.xlogid, hdr->xlp_pageaddr.xrecoff,
-                                               readId, readSeg, readOff)));
+                                               XLogFileNameP(curFileTLI, readSegNo),
+                                               readOff)));
                return false;
        }
 
@@ -4200,9 +4147,10 @@ ValidXLOGHeader(XLogPageHeader hdr, int emode)
        if (!list_member_int(expectedTLIs, (int) hdr->xlp_tli))
        {
                ereport(emode_for_corrupt_record(emode, recaddr),
-                               (errmsg("unexpected timeline ID %u in log file %u, segment %u, offset %u",
+                               (errmsg("unexpected timeline ID %u in log segment %s, offset %u",
                                                hdr->xlp_tli,
-                                               readId, readSeg, readOff)));
+                                               XLogFileNameP(curFileTLI, readSegNo),
+                                               readOff)));
                return false;
        }
 
@@ -4218,9 +4166,10 @@ ValidXLOGHeader(XLogPageHeader hdr, int emode)
        if (hdr->xlp_tli < lastPageTLI)
        {
                ereport(emode_for_corrupt_record(emode, recaddr),
-                               (errmsg("out-of-sequence timeline ID %u (after %u) in log file %u, segment %u, offset %u",
+                               (errmsg("out-of-sequence timeline ID %u (after %u) in log segment %s, offset %u",
                                                hdr->xlp_tli, lastPageTLI,
-                                               readId, readSeg, readOff)));
+                                               XLogFileNameP(curFileTLI, readSegNo),
+                                               readOff)));
                return false;
        }
        lastPageTLI = hdr->xlp_tli;
@@ -4467,7 +4416,7 @@ findNewestTimeLine(TimeLineID startTLI)
  */
 static void
 writeTimeLineHistory(TimeLineID newTLI, TimeLineID parentTLI,
-                                        TimeLineID endTLI, uint32 endLogId, uint32 endLogSeg)
+                                        TimeLineID endTLI, XLogSegNo endLogSegNo)
 {
        char            path[MAXPGPATH];
        char            tmppath[MAXPGPATH];
@@ -4557,7 +4506,7 @@ writeTimeLineHistory(TimeLineID newTLI, TimeLineID parentTLI,
         * If we did have a parent file, insert an extra newline just in case the
         * parent file failed to end with one.
         */
-       XLogFileName(xlogfname, endTLI, endLogId, endLogSeg);
+       XLogFileName(xlogfname, endTLI, endLogSegNo);
 
        /*
         * Write comment to history file to explain why and where timeline
@@ -5243,7 +5192,7 @@ BootStrapXLOG(void)
 
        /* Create first XLOG segment file */
        use_existent = false;
-       openLogFile = XLogFileInit(0, 1, &use_existent, false);
+       openLogFile = XLogFileInit(1, &use_existent, false);
 
        /* Write the first page with the initial record */
        errno = 0;
@@ -5554,7 +5503,7 @@ readRecoveryCommandFile(void)
  * Exit archive-recovery state
  */
 static void
-exitArchiveRecovery(TimeLineID endTLI, uint32 endLogId, uint32 endLogSeg)
+exitArchiveRecovery(TimeLineID endTLI, XLogSegNo endLogSegNo)
 {
        char            recoveryPath[MAXPGPATH];
        char            xlogpath[MAXPGPATH];
@@ -5590,12 +5539,11 @@ exitArchiveRecovery(TimeLineID endTLI, uint32 endLogId, uint32 endLogSeg)
         */
        if (endTLI != ThisTimeLineID)
        {
-               XLogFileCopy(endLogId, endLogSeg,
-                                        endTLI, endLogId, endLogSeg);
+               XLogFileCopy(endLogSegNo, endTLI, endLogSegNo);
 
                if (XLogArchivingActive())
                {
-                       XLogFileName(xlogpath, endTLI, endLogId, endLogSeg);
+                       XLogFileName(xlogpath, endTLI, endLogSegNo);
                        XLogArchiveNotify(xlogpath);
                }
        }
@@ -5604,7 +5552,7 @@ exitArchiveRecovery(TimeLineID endTLI, uint32 endLogId, uint32 endLogSeg)
         * Let's just make real sure there are not .ready or .done flags posted
         * for the new segment.
         */
-       XLogFileName(xlogpath, ThisTimeLineID, endLogId, endLogSeg);
+       XLogFileName(xlogpath, ThisTimeLineID, endLogSegNo);
        XLogArchiveCleanup(xlogpath);
 
        /*
@@ -6004,8 +5952,7 @@ StartupXLOG(void)
        XLogRecPtr      RecPtr,
                                checkPointLoc,
                                EndOfLog;
-       uint32          endLogId;
-       uint32          endLogSeg;
+       XLogSegNo       endLogSegNo;
        XLogRecord *record;
        uint32          freespace;
        TransactionId oldestActiveXID;
@@ -6732,7 +6679,7 @@ StartupXLOG(void)
         */
        record = ReadRecord(&LastRec, PANIC, false);
        EndOfLog = EndRecPtr;
-       XLByteToPrevSeg(EndOfLog, endLogId, endLogSeg);
+       XLByteToPrevSeg(EndOfLog, endLogSegNo);
 
        /*
         * Complain if we did not roll forward far enough to render the backup
@@ -6797,7 +6744,7 @@ StartupXLOG(void)
                ereport(LOG,
                                (errmsg("selected new timeline ID: %u", ThisTimeLineID)));
                writeTimeLineHistory(ThisTimeLineID, recoveryTargetTLI,
-                                                        curFileTLI, endLogId, endLogSeg);
+                                                        curFileTLI, endLogSegNo);
        }
 
        /* Save the selected TimeLineID in shared memory, too */
@@ -6810,20 +6757,19 @@ StartupXLOG(void)
         * we will use that below.)
         */
        if (InArchiveRecovery)
-               exitArchiveRecovery(curFileTLI, endLogId, endLogSeg);
+               exitArchiveRecovery(curFileTLI, endLogSegNo);
 
        /*
         * Prepare to write WAL starting at EndOfLog position, and init xlog
         * buffer cache using the block containing the last record from the
         * previous incarnation.
         */
-       openLogId = endLogId;
-       openLogSeg = endLogSeg;
-       openLogFile = XLogFileOpen(openLogId, openLogSeg);
+       openLogSegNo = endLogSegNo;
+       openLogFile = XLogFileOpen(openLogSegNo);
        openLogOff = 0;
        Insert = &XLogCtl->Insert;
        Insert->PrevRecord = LastRec;
-       XLogCtl->xlblocks[0].xlogid = openLogId;
+       XLogCtl->xlblocks[0].xlogid = (openLogSegNo * XLOG_SEG_SIZE) >> 32;
        XLogCtl->xlblocks[0].xrecoff =
                ((EndOfLog.xrecoff - 1) / XLOG_BLCKSZ + 1) * XLOG_BLCKSZ;
 
@@ -7644,8 +7590,7 @@ CreateCheckPoint(int flags)
        XLogCtlInsert *Insert = &XLogCtl->Insert;
        XLogRecData rdata;
        uint32          freespace;
-       uint32          _logId;
-       uint32          _logSeg;
+       XLogSegNo       _logSegNo;
        TransactionId *inCommitXids;
        int                     nInCommit;
 
@@ -7948,7 +7893,7 @@ CreateCheckPoint(int flags)
         * Select point at which we can truncate the log, which we base on the
         * prior checkpoint's earliest info.
         */
-       XLByteToSeg(ControlFile->checkPointCopy.redo, _logId, _logSeg);
+       XLByteToSeg(ControlFile->checkPointCopy.redo, _logSegNo);
 
        /*
         * Update the control file.
@@ -7991,11 +7936,11 @@ CreateCheckPoint(int flags)
         * Delete old log files (those no longer needed even for previous
         * checkpoint or the standbys in XLOG streaming).
         */
-       if (_logId || _logSeg)
+       if (_logSegNo)
        {
-               KeepLogSeg(recptr, &_logId, &_logSeg);
-               PrevLogSeg(_logId, _logSeg);
-               RemoveOldXlogFiles(_logId, _logSeg, recptr);
+               KeepLogSeg(recptr, &_logSegNo);
+               _logSegNo--;
+               RemoveOldXlogFiles(_logSegNo, recptr);
        }
 
        /*
@@ -8127,8 +8072,7 @@ CreateRestartPoint(int flags)
 {
        XLogRecPtr      lastCheckPointRecPtr;
        CheckPoint      lastCheckPoint;
-       uint32          _logId;
-       uint32          _logSeg;
+       XLogSegNo       _logSegNo;
        TimestampTz xtime;
 
        /* use volatile pointer to prevent code rearrangement */
@@ -8226,7 +8170,7 @@ CreateRestartPoint(int flags)
         * Select point at which we can truncate the xlog, which we base on the
         * prior checkpoint's earliest info.
         */
-       XLByteToSeg(ControlFile->checkPointCopy.redo, _logId, _logSeg);
+       XLByteToSeg(ControlFile->checkPointCopy.redo, _logSegNo);
 
        /*
         * Update pg_control, using current time.  Check that it still shows
@@ -8253,16 +8197,16 @@ CreateRestartPoint(int flags)
         * checkpoint/restartpoint) to prevent the disk holding the xlog from
         * growing full.
         */
-       if (_logId || _logSeg)
+       if (_logSegNo)
        {
                XLogRecPtr      endptr;
 
                /* Get the current (or recent) end of xlog */
                endptr = GetStandbyFlushRecPtr();
 
-               KeepLogSeg(endptr, &_logId, &_logSeg);
-               PrevLogSeg(_logId, _logSeg);
-               RemoveOldXlogFiles(_logId, _logSeg, endptr);
+               KeepLogSeg(endptr, &_logSegNo);
+               _logSegNo--;
+               RemoveOldXlogFiles(_logSegNo, endptr);
 
                /*
                 * Make more log segments if needed.  (Do this after recycling old log
@@ -8310,42 +8254,24 @@ CreateRestartPoint(int flags)
  * the given xlog location, recptr.
  */
 static void
-KeepLogSeg(XLogRecPtr recptr, uint32 *logId, uint32 *logSeg)
+KeepLogSeg(XLogRecPtr recptr, XLogSegNo *logSegNo)
 {
-       uint32          log;
-       uint32          seg;
-       int                     d_log;
-       int                     d_seg;
+       XLogSegNo       segno;
 
        if (wal_keep_segments == 0)
                return;
 
-       XLByteToSeg(recptr, log, seg);
+       XLByteToSeg(recptr, segno);
 
-       d_seg = wal_keep_segments % XLogSegsPerFile;
-       d_log = wal_keep_segments / XLogSegsPerFile;
-       if (seg < d_seg)
-       {
-               d_log += 1;
-               seg = seg - d_seg + XLogSegsPerFile;
-       }
-       else
-               seg = seg - d_seg;
-       /* avoid underflow, don't go below (0,1) */
-       if (log < d_log || (log == d_log && seg == 0))
-       {
-               log = 0;
-               seg = 1;
-       }
+       /* avoid underflow, don't go below 1 */
+       if (segno <= wal_keep_segments)
+               segno = 1;
        else
-               log = log - d_log;
+               segno = *logSegNo - wal_keep_segments;
 
        /* don't delete WAL segments newer than the calculated segment */
-       if (log < *logId || (log == *logId && seg < *logSeg))
-       {
-               *logId = log;
-               *logSeg = seg;
-       }
+       if (segno < *logSegNo)
+               *logSegNo = segno;
 }
 
 /*
@@ -9010,8 +8936,8 @@ assign_xlog_sync_method(int new_sync_method, void *extra)
                        if (pg_fsync(openLogFile) != 0)
                                ereport(PANIC,
                                                (errcode_for_file_access(),
-                                                errmsg("could not fsync log file %u, segment %u: %m",
-                                                               openLogId, openLogSeg)));
+                                                errmsg("could not fsync log segment %s: %m",
+                                                               XLogFileNameP(curFileTLI, readSegNo))));
                        if (get_sync_bit(sync_method) != get_sync_bit(new_sync_method))
                                XLogFileClose();
                }
@@ -9026,7 +8952,7 @@ assign_xlog_sync_method(int new_sync_method, void *extra)
  * 'log' and 'seg' are for error reporting purposes.
  */
 void
-issue_xlog_fsync(int fd, uint32 log, uint32 seg)
+issue_xlog_fsync(int fd, XLogSegNo segno)
 {
        switch (sync_method)
        {
@@ -9034,16 +8960,16 @@ issue_xlog_fsync(int fd, uint32 log, uint32 seg)
                        if (pg_fsync_no_writethrough(fd) != 0)
                                ereport(PANIC,
                                                (errcode_for_file_access(),
-                                                errmsg("could not fsync log file %u, segment %u: %m",
-                                                               log, seg)));
+                                                errmsg("could not fsync log file %s: %m",
+                                                               XLogFileNameP(ThisTimeLineID, openLogSegNo))));
                        break;
 #ifdef HAVE_FSYNC_WRITETHROUGH
                case SYNC_METHOD_FSYNC_WRITETHROUGH:
                        if (pg_fsync_writethrough(fd) != 0)
                                ereport(PANIC,
                                                (errcode_for_file_access(),
-                                                errmsg("could not fsync write-through log file %u, segment %u: %m",
-                                                               log, seg)));
+                                                errmsg("could not fsync write-through log file %s: %m",
+                                                               XLogFileNameP(ThisTimeLineID, openLogSegNo)))));
                        break;
 #endif
 #ifdef HAVE_FDATASYNC
@@ -9051,8 +8977,8 @@ issue_xlog_fsync(int fd, uint32 log, uint32 seg)
                        if (pg_fdatasync(fd) != 0)
                                ereport(PANIC,
                                                (errcode_for_file_access(),
-                                       errmsg("could not fdatasync log file %u, segment %u: %m",
-                                                  log, seg)));
+                                                errmsg("could not fdatasync log file %s: %m",
+                                                               XLogFileNameP(ThisTimeLineID, openLogSegNo))));
                        break;
 #endif
                case SYNC_METHOD_OPEN:
@@ -9065,6 +8991,17 @@ issue_xlog_fsync(int fd, uint32 log, uint32 seg)
        }
 }
 
+/*
+ * Return the filename of given log segment, as a palloc'd string.
+ */
+char *
+XLogFileNameP(TimeLineID tli, XLogSegNo segno)
+{
+       char       *result = palloc(MAXFNAMELEN);
+       XLogFileName(result, tli, segno);
+       return result;
+}
+
 /*
  * do_pg_start_backup is the workhorse of the user-visible pg_start_backup()
  * function. It creates the necessary starting checkpoint and constructs the
@@ -9096,8 +9033,7 @@ do_pg_start_backup(const char *backupidstr, bool fast, char **labelfile)
        pg_time_t       stamp_time;
        char            strfbuf[128];
        char            xlogfilename[MAXFNAMELEN];
-       uint32          _logId;
-       uint32          _logSeg;
+       XLogSegNo       _logSegNo;
        struct stat stat_buf;
        FILE       *fp;
        StringInfoData labelfbuf;
@@ -9293,8 +9229,8 @@ do_pg_start_backup(const char *backupidstr, bool fast, char **labelfile)
                        LWLockRelease(WALInsertLock);
                } while (!gotUniqueStartpoint);
 
-               XLByteToSeg(startpoint, _logId, _logSeg);
-               XLogFileName(xlogfilename, ThisTimeLineID, _logId, _logSeg);
+               XLByteToSeg(startpoint, _logSegNo);
+               XLogFileName(xlogfilename, ThisTimeLineID, _logSegNo);
 
                /*
                 * Construct backup label file
@@ -9420,8 +9356,7 @@ do_pg_stop_backup(char *labelfile, bool waitforarchive)
        char            lastxlogfilename[MAXFNAMELEN];
        char            histfilename[MAXFNAMELEN];
        char            backupfrom[20];
-       uint32          _logId;
-       uint32          _logSeg;
+       XLogSegNo       _logSegNo;
        FILE       *lfp;
        FILE       *fp;
        char            ch;
@@ -9632,8 +9567,8 @@ do_pg_stop_backup(char *labelfile, bool waitforarchive)
         */
        RequestXLogSwitch();
 
-       XLByteToPrevSeg(stoppoint, _logId, _logSeg);
-       XLogFileName(stopxlogfilename, ThisTimeLineID, _logId, _logSeg);
+       XLByteToPrevSeg(stoppoint, _logSegNo);
+       XLogFileName(stopxlogfilename, ThisTimeLineID, _logSegNo);
 
        /* Use the log timezone here, not the session timezone */
        stamp_time = (pg_time_t) time(NULL);
@@ -9644,8 +9579,8 @@ do_pg_stop_backup(char *labelfile, bool waitforarchive)
        /*
         * Write the backup history file
         */
-       XLByteToSeg(startpoint, _logId, _logSeg);
-       BackupHistoryFilePath(histfilepath, ThisTimeLineID, _logId, _logSeg,
+       XLByteToSeg(startpoint, _logSegNo);
+       BackupHistoryFilePath(histfilepath, ThisTimeLineID, _logSegNo,
                                                  startpoint.xrecoff % XLogSegSize);
        fp = AllocateFile(histfilepath, "w");
        if (!fp)
@@ -9694,11 +9629,11 @@ do_pg_stop_backup(char *labelfile, bool waitforarchive)
         */
        if (waitforarchive && XLogArchivingActive())
        {
-               XLByteToPrevSeg(stoppoint, _logId, _logSeg);
-               XLogFileName(lastxlogfilename, ThisTimeLineID, _logId, _logSeg);
+               XLByteToPrevSeg(stoppoint, _logSegNo);
+               XLogFileName(lastxlogfilename, ThisTimeLineID, _logSegNo);
 
-               XLByteToSeg(startpoint, _logId, _logSeg);
-               BackupHistoryFileName(histfilename, ThisTimeLineID, _logId, _logSeg,
+               XLByteToSeg(startpoint, _logSegNo);
+               BackupHistoryFileName(histfilename, ThisTimeLineID, _logSegNo,
                                                          startpoint.xrecoff % XLogSegSize);
 
                seconds_before_warning = 60;
@@ -10036,16 +9971,15 @@ XLogPageRead(XLogRecPtr *RecPtr, int emode, bool fetching_ckpt,
        bool            switched_segment = false;
        uint32          targetPageOff;
        uint32          targetRecOff;
-       uint32          targetId;
-       uint32          targetSeg;
+       XLogSegNo       targetSegNo;
        static pg_time_t last_fail_time = 0;
 
-       XLByteToSeg(*RecPtr, targetId, targetSeg);
+       XLByteToSeg(*RecPtr, targetSegNo);
        targetPageOff = ((RecPtr->xrecoff % XLogSegSize) / XLOG_BLCKSZ) * XLOG_BLCKSZ;
        targetRecOff = RecPtr->xrecoff % XLOG_BLCKSZ;
 
        /* Fast exit if we have read the record in the current buffer already */
-       if (failedSources == 0 && targetId == readId && targetSeg == readSeg &&
+       if (failedSources == 0 && targetSegNo == readSegNo &&
                targetPageOff == readOff && targetRecOff < readLen)
                return true;
 
@@ -10053,7 +9987,7 @@ XLogPageRead(XLogRecPtr *RecPtr, int emode, bool fetching_ckpt,
         * See if we need to switch to a new segment because the requested record
         * is not in the currently open one.
         */
-       if (readFile >= 0 && !XLByteInSeg(*RecPtr, readId, readSeg))
+       if (readFile >= 0 && !XLByteInSeg(*RecPtr, readSegNo))
        {
                /*
                 * Request a restartpoint if we've replayed too much xlog since the
@@ -10061,10 +9995,10 @@ XLogPageRead(XLogRecPtr *RecPtr, int emode, bool fetching_ckpt,
                 */
                if (StandbyMode && bgwriterLaunched)
                {
-                       if (XLogCheckpointNeeded(readId, readSeg))
+                       if (XLogCheckpointNeeded(readSegNo))
                        {
                                (void) GetRedoRecPtr();
-                               if (XLogCheckpointNeeded(readId, readSeg))
+                               if (XLogCheckpointNeeded(readSegNo))
                                        RequestCheckpoint(CHECKPOINT_CAUSE_XLOG);
                        }
                }
@@ -10074,7 +10008,7 @@ XLogPageRead(XLogRecPtr *RecPtr, int emode, bool fetching_ckpt,
                readSource = 0;
        }
 
-       XLByteToSeg(*RecPtr, readId, readSeg);
+       XLByteToSeg(*RecPtr, readSegNo);
 
 retry:
        /* See if we need to retrieve more data */
@@ -10152,7 +10086,7 @@ retry:
                                                if (readFile < 0)
                                                {
                                                        readFile =
-                                                               XLogFileRead(readId, readSeg, PANIC,
+                                                               XLogFileRead(readSegNo, PANIC,
                                                                                         recoveryTargetTLI,
                                                                                         XLOG_FROM_STREAM, false);
                                                        Assert(readFile >= 0);
@@ -10258,7 +10192,7 @@ retry:
                                        }
                                        /* Don't try to read from a source that just failed */
                                        sources &= ~failedSources;
-                                       readFile = XLogFileReadAnyTLI(readId, readSeg, DEBUG2,
+                                       readFile = XLogFileReadAnyTLI(readSegNo, DEBUG2,
                                                                                                  sources);
                                        switched_segment = true;
                                        if (readFile >= 0)
@@ -10301,8 +10235,7 @@ retry:
                                if (InArchiveRecovery)
                                        sources |= XLOG_FROM_ARCHIVE;
 
-                               readFile = XLogFileReadAnyTLI(readId, readSeg, emode,
-                                                                                         sources);
+                               readFile = XLogFileReadAnyTLI(readSegNo, emode, sources);
                                switched_segment = true;
                                if (readFile < 0)
                                        return false;
@@ -10347,10 +10280,12 @@ retry:
                readOff = 0;
                if (read(readFile, readBuf, XLOG_BLCKSZ) != XLOG_BLCKSZ)
                {
+                       char fname[MAXFNAMELEN];
+                       XLogFileName(fname, curFileTLI, readSegNo);
                        ereport(emode_for_corrupt_record(emode, *RecPtr),
                                        (errcode_for_file_access(),
-                                        errmsg("could not read from log file %u, segment %u, offset %u: %m",
-                                                       readId, readSeg, readOff)));
+                                        errmsg("could not read from log segment %s, offset %u: %m",
+                                                       fname, readOff)));
                        goto next_record_is_invalid;
                }
                if (!ValidXLOGHeader((XLogPageHeader) readBuf, emode))
@@ -10361,25 +10296,28 @@ retry:
        readOff = targetPageOff;
        if (lseek(readFile, (off_t) readOff, SEEK_SET) < 0)
        {
+               char fname[MAXFNAMELEN];
+               XLogFileName(fname, curFileTLI, readSegNo);
                ereport(emode_for_corrupt_record(emode, *RecPtr),
                                (errcode_for_file_access(),
-                errmsg("could not seek in log file %u, segment %u to offset %u: %m",
-                               readId, readSeg, readOff)));
+                errmsg("could not seek in log segment %s to offset %u: %m",
+                               fname, readOff)));
                goto next_record_is_invalid;
        }
        if (read(readFile, readBuf, XLOG_BLCKSZ) != XLOG_BLCKSZ)
        {
+               char fname[MAXFNAMELEN];
+               XLogFileName(fname, curFileTLI, readSegNo);
                ereport(emode_for_corrupt_record(emode, *RecPtr),
                                (errcode_for_file_access(),
-                errmsg("could not read from log file %u, segment %u, offset %u: %m",
-                               readId, readSeg, readOff)));
+                errmsg("could not read from log segment %s, offset %u: %m",
+                               fname, readOff)));
                goto next_record_is_invalid;
        }
        if (!ValidXLOGHeader((XLogPageHeader) readBuf, emode))
                goto next_record_is_invalid;
 
-       Assert(targetId == readId);
-       Assert(targetSeg == readSeg);
+       Assert(targetSegNo == readSegNo);
        Assert(targetPageOff == readOff);
        Assert(targetRecOff < readLen);
 
index adf0b16e56a052db68196d33fc7236e7bdbf7218..bbb87d4d5cd847ed0f025b55ae3d2f248d42ecfb 100644 (file)
@@ -271,8 +271,7 @@ pg_xlogfile_name_offset(PG_FUNCTION_ARGS)
        char       *locationstr;
        unsigned int uxlogid;
        unsigned int uxrecoff;
-       uint32          xlogid;
-       uint32          xlogseg;
+       XLogSegNo       xlogsegno;
        uint32          xrecoff;
        XLogRecPtr      locationpoint;
        char            xlogfilename[MAXFNAMELEN];
@@ -319,8 +318,8 @@ pg_xlogfile_name_offset(PG_FUNCTION_ARGS)
        /*
         * xlogfilename
         */
-       XLByteToPrevSeg(locationpoint, xlogid, xlogseg);
-       XLogFileName(xlogfilename, ThisTimeLineID, xlogid, xlogseg);
+       XLByteToPrevSeg(locationpoint, xlogsegno);
+       XLogFileName(xlogfilename, ThisTimeLineID, xlogsegno);
 
        values[0] = CStringGetTextDatum(xlogfilename);
        isnull[0] = false;
@@ -328,7 +327,7 @@ pg_xlogfile_name_offset(PG_FUNCTION_ARGS)
        /*
         * offset
         */
-       xrecoff = locationpoint.xrecoff - xlogseg * XLogSegSize;
+       xrecoff = locationpoint.xrecoff % XLogSegSize;
 
        values[1] = UInt32GetDatum(xrecoff);
        isnull[1] = false;
@@ -354,8 +353,7 @@ pg_xlogfile_name(PG_FUNCTION_ARGS)
        char       *locationstr;
        unsigned int uxlogid;
        unsigned int uxrecoff;
-       uint32          xlogid;
-       uint32          xlogseg;
+       XLogSegNo       xlogsegno;
        XLogRecPtr      locationpoint;
        char            xlogfilename[MAXFNAMELEN];
 
@@ -378,8 +376,8 @@ pg_xlogfile_name(PG_FUNCTION_ARGS)
        locationpoint.xlogid = uxlogid;
        locationpoint.xrecoff = uxrecoff;
 
-       XLByteToPrevSeg(locationpoint, xlogid, xlogseg);
-       XLogFileName(xlogfilename, ThisTimeLineID, xlogid, xlogseg);
+       XLByteToPrevSeg(locationpoint, xlogsegno);
+       XLogFileName(xlogfilename, ThisTimeLineID, xlogsegno);
 
        PG_RETURN_TEXT_P(cstring_to_text(xlogfilename));
 }
@@ -514,6 +512,8 @@ pg_xlog_location_diff(PG_FUNCTION_ARGS)
        XLogRecPtr      loc1,
                                loc2;
        Numeric         result;
+       uint64          bytes1,
+                               bytes2;
 
        /*
         * Read and parse input
@@ -533,33 +533,17 @@ pg_xlog_location_diff(PG_FUNCTION_ARGS)
                                (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
                   errmsg("could not parse transaction log location \"%s\"", str2)));
 
-       /*
-        * Sanity check
-        */
-       if (loc1.xrecoff > XLogFileSize)
-               ereport(ERROR,
-                               (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-                                errmsg("xrecoff \"%X\" is out of valid range, 0..%X", loc1.xrecoff, XLogFileSize)));
-       if (loc2.xrecoff > XLogFileSize)
-               ereport(ERROR,
-                               (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-                                errmsg("xrecoff \"%X\" is out of valid range, 0..%X", loc2.xrecoff, XLogFileSize)));
+       bytes1 = (((uint64)loc1.xlogid) << 32L) + loc1.xrecoff;
+       bytes2 = (((uint64)loc2.xlogid) << 32L) + loc2.xrecoff;
 
        /*
-        * result = XLogFileSize * (xlogid1 - xlogid2) + xrecoff1 - xrecoff2
+        * result = bytes1 - bytes2.
+        *
+        * XXX: this won't handle values higher than 2^63 correctly.
         */
        result = DatumGetNumeric(DirectFunctionCall2(numeric_sub,
-          DirectFunctionCall1(int8_numeric, Int64GetDatum((int64) loc1.xlogid)),
-        DirectFunctionCall1(int8_numeric, Int64GetDatum((int64) loc2.xlogid))));
-       result = DatumGetNumeric(DirectFunctionCall2(numeric_mul,
-         DirectFunctionCall1(int8_numeric, Int64GetDatum((int64) XLogFileSize)),
-                                                                                                NumericGetDatum(result)));
-       result = DatumGetNumeric(DirectFunctionCall2(numeric_add,
-                                                                                                NumericGetDatum(result),
-       DirectFunctionCall1(int8_numeric, Int64GetDatum((int64) loc1.xrecoff))));
-       result = DatumGetNumeric(DirectFunctionCall2(numeric_sub,
-                                                                                                NumericGetDatum(result),
-       DirectFunctionCall1(int8_numeric, Int64GetDatum((int64) loc2.xrecoff))));
+          DirectFunctionCall1(int8_numeric, Int64GetDatum((int64) bytes1)),
+          DirectFunctionCall1(int8_numeric, Int64GetDatum((int64) bytes2))));
 
        PG_RETURN_NUMERIC(result);
 }
index 87aa4e1ba0a2082eb47175f16bc321d058cbee99..7e5db863daf254f0431d6b1bccfef21959b78af9 100644 (file)
@@ -779,7 +779,7 @@ IsCheckpointOnSchedule(double progress)
        {
                recptr = GetInsertRecPtr();
                elapsed_xlogs =
-                       (((double) (int32) (recptr.xlogid - ckpt_start_recptr.xlogid)) * XLogSegsPerFile +
+                       (((double) ((uint64) (recptr.xlogid - ckpt_start_recptr.xlogid) << 32L)) +
                         ((double) recptr.xrecoff - (double) ckpt_start_recptr.xrecoff) / XLogSegSize) /
                        CheckPointSegments;
 
index 0bc88a4040d6938c4619bb6ad9c42a6deea9a03e..14c42b46c23b2ea4ed282ad7fb9402e33a730724 100644 (file)
@@ -221,10 +221,8 @@ perform_base_backup(basebackup_options *opt, DIR *tblspcdir)
                 * We've left the last tar file "open", so we can now append the
                 * required WAL files to it.
                 */
-               uint32          logid,
-                                       logseg;
-               uint32          endlogid,
-                                       endlogseg;
+               XLogSegNo       logsegno;
+               XLogSegNo       endlogsegno;
                struct stat statbuf;
 
                MemSet(&statbuf, 0, sizeof(statbuf));
@@ -236,8 +234,8 @@ perform_base_backup(basebackup_options *opt, DIR *tblspcdir)
                statbuf.st_size = XLogSegSize;
                statbuf.st_mtime = time(NULL);
 
-               XLByteToSeg(startptr, logid, logseg);
-               XLByteToPrevSeg(endptr, endlogid, endlogseg);
+               XLByteToSeg(startptr, logsegno);
+               XLByteToPrevSeg(endptr, endlogsegno);
 
                while (true)
                {
@@ -245,7 +243,7 @@ perform_base_backup(basebackup_options *opt, DIR *tblspcdir)
                        char            fn[MAXPGPATH];
                        int                     i;
 
-                       XLogFilePath(fn, ThisTimeLineID, logid, logseg);
+                       XLogFilePath(fn, ThisTimeLineID, logsegno);
                        _tarWriteHeader(fn, NULL, &statbuf);
 
                        /* Send the actual WAL file contents, block-by-block */
@@ -254,8 +252,7 @@ perform_base_backup(basebackup_options *opt, DIR *tblspcdir)
                                char            buf[TAR_SEND_SIZE];
                                XLogRecPtr      ptr;
 
-                               ptr.xlogid = logid;
-                               ptr.xrecoff = logseg * XLogSegSize + TAR_SEND_SIZE * i;
+                               XLogSegNoOffsetToRecPtr(logsegno, TAR_SEND_SIZE * i, ptr);
 
                                /*
                                 * Some old compilers, e.g. gcc 2.95.3/x86, think that passing
@@ -277,11 +274,10 @@ perform_base_backup(basebackup_options *opt, DIR *tblspcdir)
 
 
                        /* Advance to the next WAL file */
-                       NextLogSeg(logid, logseg);
+                       logsegno++;
 
                        /* Have we reached our stop position yet? */
-                       if (logid > endlogid ||
-                               (logid == endlogid && logseg > endlogseg))
+                       if (logsegno > endlogsegno)
                                break;
                }
 
index 650b74fff7d5ca5b6201abfce8efffd332f04d64..b3ba7089dfb39927744437759e8989bb84008d8f 100644 (file)
@@ -69,11 +69,12 @@ walrcv_disconnect_type walrcv_disconnect = NULL;
 
 /*
  * These variables are used similarly to openLogFile/Id/Seg/Off,
- * but for walreceiver to write the XLOG.
+ * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
+ * corresponding the filename of recvFile, used for error messages.
  */
 static int     recvFile = -1;
-static uint32 recvId = 0;
-static uint32 recvSeg = 0;
+static TimeLineID      recvFileTLI = -1;
+static XLogSegNo recvSegNo = 0;
 static uint32 recvOff = 0;
 
 /*
@@ -481,7 +482,7 @@ XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr)
        {
                int                     segbytes;
 
-               if (recvFile < 0 || !XLByteInSeg(recptr, recvId, recvSeg))
+               if (recvFile < 0 || !XLByteInSeg(recptr, recvSegNo))
                {
                        bool            use_existent;
 
@@ -501,15 +502,16 @@ XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr)
                                if (close(recvFile) != 0)
                                        ereport(PANIC,
                                                        (errcode_for_file_access(),
-                                               errmsg("could not close log file %u, segment %u: %m",
-                                                          recvId, recvSeg)));
+                                                        errmsg("could not close log segment %s: %m",
+                                                                       XLogFileNameP(recvFileTLI, recvSegNo))));
                        }
                        recvFile = -1;
 
                        /* Create/use new log file */
-                       XLByteToSeg(recptr, recvId, recvSeg);
+                       XLByteToSeg(recptr, recvSegNo);
                        use_existent = true;
-                       recvFile = XLogFileInit(recvId, recvSeg, &use_existent, true);
+                       recvFile = XLogFileInit(recvSegNo, &use_existent, true);
+                       recvFileTLI = ThisTimeLineID;
                        recvOff = 0;
                }
 
@@ -527,9 +529,9 @@ XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr)
                        if (lseek(recvFile, (off_t) startoff, SEEK_SET) < 0)
                                ereport(PANIC,
                                                (errcode_for_file_access(),
-                                                errmsg("could not seek in log file %u, "
-                                                               "segment %u to offset %u: %m",
-                                                               recvId, recvSeg, startoff)));
+                                                errmsg("could not seek in log segment %s, to offset %u: %m",
+                                                               XLogFileNameP(recvFileTLI, recvSegNo),
+                                                               startoff)));
                        recvOff = startoff;
                }
 
@@ -544,9 +546,9 @@ XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr)
                                errno = ENOSPC;
                        ereport(PANIC,
                                        (errcode_for_file_access(),
-                                        errmsg("could not write to log file %u, segment %u "
+                                        errmsg("could not write to log segment %s "
                                                        "at offset %u, length %lu: %m",
-                                                       recvId, recvSeg,
+                                                       XLogFileNameP(recvFileTLI, recvSegNo),
                                                        recvOff, (unsigned long) segbytes)));
                }
 
@@ -575,7 +577,7 @@ XLogWalRcvFlush(bool dying)
                /* use volatile pointer to prevent code rearrangement */
                volatile WalRcvData *walrcv = WalRcv;
 
-               issue_xlog_fsync(recvFile, recvId, recvSeg);
+               issue_xlog_fsync(recvFile, recvSegNo);
 
                LogstreamResult.Flush = LogstreamResult.Write;
 
index 45a3b2ef2941063d9901d1f060a73c2e3897c751..2c04df08ed1874499543bdb410b8d45dfa357774 100644 (file)
@@ -87,8 +87,7 @@ int                   replication_timeout = 60 * 1000;        /* maximum time to send one
  * but for walsender to read the XLOG.
  */
 static int     sendFile = -1;
-static uint32 sendId = 0;
-static uint32 sendSeg = 0;
+static XLogSegNo sendSegNo = 0;
 static uint32 sendOff = 0;
 
 /*
@@ -977,10 +976,8 @@ XLogRead(char *buf, XLogRecPtr startptr, Size count)
        char       *p;
        XLogRecPtr      recptr;
        Size            nbytes;
-       uint32          lastRemovedLog;
-       uint32          lastRemovedSeg;
-       uint32          log;
-       uint32          seg;
+       XLogSegNo       lastRemovedSegNo;
+       XLogSegNo       segno;
 
 retry:
        p = buf;
@@ -995,7 +992,7 @@ retry:
 
                startoff = recptr.xrecoff % XLogSegSize;
 
-               if (sendFile < 0 || !XLByteInSeg(recptr, sendId, sendSeg))
+               if (sendFile < 0 || !XLByteInSeg(recptr, sendSegNo))
                {
                        char            path[MAXPGPATH];
 
@@ -1003,8 +1000,8 @@ retry:
                        if (sendFile >= 0)
                                close(sendFile);
 
-                       XLByteToSeg(recptr, sendId, sendSeg);
-                       XLogFilePath(path, ThisTimeLineID, sendId, sendSeg);
+                       XLByteToSeg(recptr, sendSegNo);
+                       XLogFilePath(path, ThisTimeLineID, sendSegNo);
 
                        sendFile = BasicOpenFile(path, O_RDONLY | PG_BINARY, 0);
                        if (sendFile < 0)
@@ -1015,20 +1012,15 @@ retry:
                                 * removed or recycled.
                                 */
                                if (errno == ENOENT)
-                               {
-                                       char            filename[MAXFNAMELEN];
-
-                                       XLogFileName(filename, ThisTimeLineID, sendId, sendSeg);
                                        ereport(ERROR,
                                                        (errcode_for_file_access(),
                                                         errmsg("requested WAL segment %s has already been removed",
-                                                                       filename)));
-                               }
+                                                                       XLogFileNameP(ThisTimeLineID, sendSegNo))));
                                else
                                        ereport(ERROR,
                                                        (errcode_for_file_access(),
-                                                        errmsg("could not open file \"%s\" (log file %u, segment %u): %m",
-                                                                       path, sendId, sendSeg)));
+                                                        errmsg("could not open file \"%s\": %m",
+                                                                       path)));
                        }
                        sendOff = 0;
                }
@@ -1039,8 +1031,9 @@ retry:
                        if (lseek(sendFile, (off_t) startoff, SEEK_SET) < 0)
                                ereport(ERROR,
                                                (errcode_for_file_access(),
-                                                errmsg("could not seek in log file %u, segment %u to offset %u: %m",
-                                                               sendId, sendSeg, startoff)));
+                                                errmsg("could not seek in log segment %s to offset %u: %m",
+                                                               XLogFileNameP(ThisTimeLineID, sendSegNo),
+                                                               startoff)));
                        sendOff = startoff;
                }
 
@@ -1052,11 +1045,13 @@ retry:
 
                readbytes = read(sendFile, p, segbytes);
                if (readbytes <= 0)
+               {
                        ereport(ERROR,
                                        (errcode_for_file_access(),
-                       errmsg("could not read from log file %u, segment %u, offset %u, "
-                                  "length %lu: %m",
-                                  sendId, sendSeg, sendOff, (unsigned long) segbytes)));
+                       errmsg("could not read from log segment %s, offset %u, length %lu: %m",
+                                  XLogFileNameP(ThisTimeLineID, sendSegNo),
+                                  sendOff, (unsigned long) segbytes)));
+               }
 
                /* Update state for read */
                XLByteAdvance(recptr, readbytes);
@@ -1073,19 +1068,13 @@ retry:
         * read() succeeds in that case, but the data we tried to read might
         * already have been overwritten with new WAL records.
         */
-       XLogGetLastRemoved(&lastRemovedLog, &lastRemovedSeg);
-       XLByteToSeg(startptr, log, seg);
-       if (log < lastRemovedLog ||
-               (log == lastRemovedLog && seg <= lastRemovedSeg))
-       {
-               char            filename[MAXFNAMELEN];
-
-               XLogFileName(filename, ThisTimeLineID, log, seg);
+       XLogGetLastRemoved(&lastRemovedSegNo);
+       XLByteToSeg(startptr, segno);
+       if (segno <= lastRemovedSegNo)
                ereport(ERROR,
                                (errcode_for_file_access(),
                                 errmsg("requested WAL segment %s has already been removed",
-                                               filename)));
-       }
+                                               XLogFileNameP(ThisTimeLineID, segno))));
 
        /*
         * During recovery, the currently-open WAL file might be replaced with the
@@ -1165,24 +1154,13 @@ XLogSend(char *msgbuf, bool *caughtup)
         * SendRqstPtr never points to the middle of a WAL record.
         */
        startptr = sentPtr;
-       if (startptr.xrecoff >= XLogFileSize)
-       {
-               /*
-                * crossing a logid boundary, skip the non-existent last log segment
-                * in previous logical log file.
-                */
-               startptr.xlogid += 1;
-               startptr.xrecoff = 0;
-       }
-
        endptr = startptr;
        XLByteAdvance(endptr, MAX_SEND_SIZE);
        if (endptr.xlogid != startptr.xlogid)
        {
                /* Don't cross a logfile boundary within one message */
                Assert(endptr.xlogid == startptr.xlogid + 1);
-               endptr.xlogid = startptr.xlogid;
-               endptr.xrecoff = XLogFileSize;
+               endptr.xrecoff = 0;
        }
 
        /* if we went beyond SendRqstPtr, back off */
@@ -1198,7 +1176,10 @@ XLogSend(char *msgbuf, bool *caughtup)
                *caughtup = false;
        }
 
-       nbytes = endptr.xrecoff - startptr.xrecoff;
+       if (endptr.xrecoff == 0)
+               nbytes = 0x100000000L - (uint64) startptr.xrecoff;
+       else
+               nbytes = endptr.xrecoff - startptr.xrecoff;
        Assert(nbytes <= MAX_SEND_SIZE);
 
        /*
index 20adb653cf1f16d60db3f30f925e77ecf4ed8449..4b109f4b96fe7cb92192e3585320d6453e5cf945 100644 (file)
@@ -102,8 +102,7 @@ FindStreamingStart(XLogRecPtr currentpos, uint32 currenttimeline)
        struct dirent *dirent;
        int                     i;
        bool            b;
-       uint32          high_log = 0;
-       uint32          high_seg = 0;
+       XLogSegNo       high_segno = 0;
 
        dir = opendir(basedir);
        if (dir == NULL)
@@ -117,9 +116,10 @@ FindStreamingStart(XLogRecPtr currentpos, uint32 currenttimeline)
        {
                char            fullpath[MAXPGPATH];
                struct stat statbuf;
-               uint32          tli,
-                                       log,
+               uint32          tli;
+               unsigned int log,
                                        seg;
+               XLogSegNo       segno;
 
                if (strcmp(dirent->d_name, ".") == 0 || strcmp(dirent->d_name, "..") == 0)
                        continue;
@@ -151,6 +151,7 @@ FindStreamingStart(XLogRecPtr currentpos, uint32 currenttimeline)
                                        progname, dirent->d_name);
                        disconnect_and_exit(1);
                }
+               segno = ((uint64) log) << 32 | seg;
 
                /* Ignore any files that are for another timeline */
                if (tli != currenttimeline)
@@ -168,11 +169,9 @@ FindStreamingStart(XLogRecPtr currentpos, uint32 currenttimeline)
                if (statbuf.st_size == XLOG_SEG_SIZE)
                {
                        /* Completed segment */
-                       if (log > high_log ||
-                               (log == high_log && seg > high_seg))
+                       if (segno > high_segno)
                        {
-                               high_log = log;
-                               high_seg = seg;
+                               high_segno = segno;
                                continue;
                        }
                }
@@ -186,7 +185,7 @@ FindStreamingStart(XLogRecPtr currentpos, uint32 currenttimeline)
 
        closedir(dir);
 
-       if (high_log > 0 || high_seg > 0)
+       if (high_segno > 0)
        {
                XLogRecPtr      high_ptr;
 
@@ -194,10 +193,9 @@ FindStreamingStart(XLogRecPtr currentpos, uint32 currenttimeline)
                 * Move the starting pointer to the start of the next segment, since
                 * the highest one we've seen was completed.
                 */
-               NextLogSeg(high_log, high_seg);
+               high_segno++;
 
-               high_ptr.xlogid = high_log;
-               high_ptr.xrecoff = high_seg * XLOG_SEG_SIZE;
+               XLogSegNoOffsetToRecPtr(high_segno, 0, high_ptr);
 
                return high_ptr;
        }
index 9dd94e1140683ee7ec2c44cb8d1866b980ff7b46..8c6755caa981c01d8be5812a4b343af508f9d621 100644 (file)
@@ -55,9 +55,10 @@ open_walfile(XLogRecPtr startpoint, uint32 timeline, char *basedir, char *namebu
        struct stat statbuf;
        char       *zerobuf;
        int                     bytes;
+       XLogSegNo       segno;
 
-       XLogFileName(namebuf, timeline, startpoint.xlogid,
-                                startpoint.xrecoff / XLOG_SEG_SIZE);
+       XLByteToSeg(startpoint, segno);
+       XLogFileName(namebuf, timeline, segno);
 
        snprintf(fn, sizeof(fn), "%s/%s.partial", basedir, namebuf);
        f = open(fn, O_WRONLY | O_CREAT | PG_BINARY, S_IRUSR | S_IWUSR);
index 5ecf5c49304692abc3702b1089586a803f2b20f4..554e08c98cb4d293032794b4255176095853c83b 100644 (file)
@@ -60,8 +60,7 @@ extern char *optarg;
 
 
 static ControlFileData ControlFile;            /* pg_control values */
-static uint32 newXlogId,
-                       newXlogSeg;                     /* ID/Segment of new XLOG segment */
+static XLogSegNo newXlogSegNo; /* new XLOG segment # */
 static bool guessed = false;   /* T if we had to guess at any values */
 static const char *progname;
 
@@ -87,12 +86,9 @@ main(int argc, char *argv[])
        Oid                     set_oid = 0;
        MultiXactId set_mxid = 0;
        MultiXactOffset set_mxoff = (MultiXactOffset) -1;
-       uint32          minXlogTli = 0,
-                               minXlogId = 0,
-                               minXlogSeg = 0;
+       uint32          minXlogTli = 0;
+       XLogSegNo       minXlogSegNo = 0;
        char       *endptr;
-       char       *endptr2;
-       char       *endptr3;
        char       *DataDir;
        int                     fd;
        char            path[MAXPGPATH];
@@ -204,27 +200,13 @@ main(int argc, char *argv[])
                                break;
 
                        case 'l':
-                               minXlogTli = strtoul(optarg, &endptr, 0);
-                               if (endptr == optarg || *endptr != ',')
-                               {
-                                       fprintf(stderr, _("%s: invalid argument for option -l\n"), progname);
-                                       fprintf(stderr, _("Try \"%s --help\" for more information.\n"), progname);
-                                       exit(1);
-                               }
-                               minXlogId = strtoul(endptr + 1, &endptr2, 0);
-                               if (endptr2 == endptr + 1 || *endptr2 != ',')
-                               {
-                                       fprintf(stderr, _("%s: invalid argument for option -l\n"), progname);
-                                       fprintf(stderr, _("Try \"%s --help\" for more information.\n"), progname);
-                                       exit(1);
-                               }
-                               minXlogSeg = strtoul(endptr2 + 1, &endptr3, 0);
-                               if (endptr3 == endptr2 + 1 || *endptr3 != '\0')
+                               if (strspn(optarg, "01234567890ABCDEFabcdef") != 24)
                                {
                                        fprintf(stderr, _("%s: invalid argument for option -l\n"), progname);
                                        fprintf(stderr, _("Try \"%s --help\" for more information.\n"), progname);
                                        exit(1);
                                }
+                               XLogFromFileName(optarg, &minXlogTli, &minXlogSegNo);
                                break;
 
                        default:
@@ -295,7 +277,7 @@ main(int argc, char *argv[])
                GuessControlValues();
 
        /*
-        * Also look at existing segment files to set up newXlogId/newXlogSeg
+        * Also look at existing segment files to set up newXlogSegNo
         */
        FindEndOfXLOG();
 
@@ -335,13 +317,8 @@ main(int argc, char *argv[])
        if (minXlogTli > ControlFile.checkPointCopy.ThisTimeLineID)
                ControlFile.checkPointCopy.ThisTimeLineID = minXlogTli;
 
-       if (minXlogId > newXlogId ||
-               (minXlogId == newXlogId &&
-                minXlogSeg > newXlogSeg))
-       {
-               newXlogId = minXlogId;
-               newXlogSeg = minXlogSeg;
-       }
+       if (minXlogSegNo > newXlogSegNo)
+               newXlogSegNo = minXlogSegNo;
 
        /*
         * If we had to guess anything, and -f was not given, just print the
@@ -545,6 +522,7 @@ static void
 PrintControlValues(bool guessed)
 {
        char            sysident_str[32];
+       char            fname[MAXFNAMELEN];
 
        if (guessed)
                printf(_("Guessed pg_control values:\n\n"));
@@ -558,10 +536,10 @@ PrintControlValues(bool guessed)
        snprintf(sysident_str, sizeof(sysident_str), UINT64_FORMAT,
                         ControlFile.system_identifier);
 
-       printf(_("First log file ID after reset:        %u\n"),
-                  newXlogId);
-       printf(_("First log file segment after reset:   %u\n"),
-                  newXlogSeg);
+       XLogFileName(fname, ControlFile.checkPointCopy.ThisTimeLineID, newXlogSegNo);
+
+       printf(_("First log segment after reset:        %s\n"),
+                  fname);
        printf(_("pg_control version number:            %u\n"),
                   ControlFile.pg_control_version);
        printf(_("Catalog version number:               %u\n"),
@@ -624,11 +602,10 @@ RewriteControlFile(void)
 
        /*
         * Adjust fields as needed to force an empty XLOG starting at
-        * newXlogId/newXlogSeg.
+        * newXlogSegNo.
         */
-       ControlFile.checkPointCopy.redo.xlogid = newXlogId;
-       ControlFile.checkPointCopy.redo.xrecoff =
-               newXlogSeg * XLogSegSize + SizeOfXLogLongPHD;
+       XLogSegNoOffsetToRecPtr(newXlogSegNo, SizeOfXLogLongPHD,
+                                                       ControlFile.checkPointCopy.redo);
        ControlFile.checkPointCopy.time = (pg_time_t) time(NULL);
 
        ControlFile.state = DB_SHUTDOWNED;
@@ -728,14 +705,17 @@ FindEndOfXLOG(void)
 {
        DIR                *xldir;
        struct dirent *xlde;
+       uint64          segs_per_xlogid;
+       uint64          xlogbytepos;
 
        /*
         * Initialize the max() computation using the last checkpoint address from
         * old pg_control.      Note that for the moment we are working with segment
         * numbering according to the old xlog seg size.
         */
-       newXlogId = ControlFile.checkPointCopy.redo.xlogid;
-       newXlogSeg = ControlFile.checkPointCopy.redo.xrecoff / ControlFile.xlog_seg_size;
+       segs_per_xlogid = (0x100000000L / ControlFile.xlog_seg_size);
+       newXlogSegNo = ((uint64) ControlFile.checkPointCopy.redo.xlogid) * segs_per_xlogid
+               + (ControlFile.checkPointCopy.redo.xrecoff / ControlFile.xlog_seg_size);
 
        /*
         * Scan the pg_xlog directory to find existing WAL segment files. We
@@ -759,8 +739,10 @@ FindEndOfXLOG(void)
                        unsigned int tli,
                                                log,
                                                seg;
+                       XLogSegNo       segno;
 
                        sscanf(xlde->d_name, "%08X%08X%08X", &tli, &log, &seg);
+                       segno = ((uint64) log) * segs_per_xlogid + seg;
 
                        /*
                         * Note: we take the max of all files found, regardless of their
@@ -768,12 +750,8 @@ FindEndOfXLOG(void)
                         * timelines other than the target TLI, but this seems safer.
                         * Better too large a result than too small...
                         */
-                       if (log > newXlogId ||
-                               (log == newXlogId && seg > newXlogSeg))
-                       {
-                               newXlogId = log;
-                               newXlogSeg = seg;
-                       }
+                       if (segno > newXlogSegNo)
+                               newXlogSegNo = segno;
                }
                errno = 0;
        }
@@ -799,11 +777,9 @@ FindEndOfXLOG(void)
         * Finally, convert to new xlog seg size, and advance by one to ensure we
         * are in virgin territory.
         */
-       newXlogSeg *= ControlFile.xlog_seg_size;
-       newXlogSeg = (newXlogSeg + XLogSegSize - 1) / XLogSegSize;
-
-       /* be sure we wrap around correctly at end of a logfile */
-       NextLogSeg(newXlogId, newXlogSeg);
+       xlogbytepos = newXlogSegNo * ControlFile.xlog_seg_size;
+       newXlogSegNo = (xlogbytepos + XLogSegSize - 1) / XLogSegSize;
+       newXlogSegNo++;
 }
 
 
@@ -972,8 +948,7 @@ WriteEmptyXLOG(void)
        record->xl_crc = crc;
 
        /* Write the first page */
-       XLogFilePath(path, ControlFile.checkPointCopy.ThisTimeLineID,
-                                newXlogId, newXlogSeg);
+       XLogFilePath(path, ControlFile.checkPointCopy.ThisTimeLineID, newXlogSegNo);
 
        unlink(path);
 
index df5f232eeea44684b25ce81673b54e4ad47cc33c..b58191085b4003f7ffe714f85b2e20e00e38e2d6 100644 (file)
@@ -267,12 +267,10 @@ extern XLogRecPtr XLogInsert(RmgrId rmid, uint8 info, XLogRecData *rdata);
 extern void XLogFlush(XLogRecPtr RecPtr);
 extern bool XLogBackgroundFlush(void);
 extern bool XLogNeedsFlush(XLogRecPtr RecPtr);
-extern int XLogFileInit(uint32 log, uint32 seg,
-                        bool *use_existent, bool use_lock);
-extern int     XLogFileOpen(uint32 log, uint32 seg);
+extern int XLogFileInit(XLogSegNo segno, bool *use_existent, bool use_lock);
+extern int     XLogFileOpen(XLogSegNo segno);
 
-
-extern void XLogGetLastRemoved(uint32 *log, uint32 *seg);
+extern void XLogGetLastRemoved(XLogSegNo *segno);
 extern void XLogSetAsyncXactLSN(XLogRecPtr record);
 
 extern void RestoreBkpBlocks(XLogRecPtr lsn, XLogRecord *record, bool cleanup);
@@ -280,7 +278,7 @@ extern void RestoreBkpBlocks(XLogRecPtr lsn, XLogRecord *record, bool cleanup);
 extern void xlog_redo(XLogRecPtr lsn, XLogRecord *record);
 extern void xlog_desc(StringInfo buf, uint8 xl_info, char *rec);
 
-extern void issue_xlog_fsync(int fd, uint32 log, uint32 seg);
+extern void issue_xlog_fsync(int fd, XLogSegNo segno);
 
 extern bool RecoveryInProgress(void);
 extern bool HotStandbyActive(void);
@@ -294,6 +292,7 @@ extern bool RecoveryIsPaused(void);
 extern void SetRecoveryPause(bool recoveryPause);
 extern TimestampTz GetLatestXTime(void);
 extern TimestampTz GetCurrentChunkReplayStartTime(void);
+extern char *XLogFileNameP(TimeLineID tli, XLogSegNo segno);
 
 extern void UpdateControlFile(void);
 extern uint64 GetSystemIdentifier(void);
index 29a0310dcf931503a8baa57951502618946247ab..50b4c2f3782194cd180c77ac01b7a9a24a9b7216 100644 (file)
@@ -71,7 +71,7 @@ typedef struct XLogContRecord
 /*
  * Each page of XLOG file has a header like this:
  */
-#define XLOG_PAGE_MAGIC 0xD071 /* can be used as WAL version indicator */
+#define XLOG_PAGE_MAGIC 0xD072 /* can be used as WAL version indicator */
 
 typedef struct XLogPageHeaderData
 {
@@ -115,55 +115,27 @@ typedef XLogLongPageHeaderData *XLogLongPageHeader;
        (((hdr)->xlp_info & XLP_LONG_HEADER) ? SizeOfXLogLongPHD : SizeOfXLogShortPHD)
 
 /*
- * We break each logical log file (xlogid value) into segment files of the
- * size indicated by XLOG_SEG_SIZE.  One possible segment at the end of each
- * log file is wasted, to ensure that we don't have problems representing
- * last-byte-position-plus-1.
+ * The XLOG is split into WAL segments (physical files) of the size indicated
+ * by XLOG_SEG_SIZE.
  */
 #define XLogSegSize            ((uint32) XLOG_SEG_SIZE)
-#define XLogSegsPerFile (((uint32) 0xffffffff) / XLogSegSize)
-#define XLogFileSize   (XLogSegsPerFile * XLogSegSize)
+#define XLogSegmentsPerXLogId  (0x100000000L / XLOG_SEG_SIZE)
 
+#define XLogSegNoOffsetToRecPtr(segno, offset, dest) \
+       do {    \
+               (dest).xlogid = (segno) / XLogSegmentsPerXLogId;                                \
+               (dest).xrecoff = ((segno) % XLogSegmentsPerXLogId) * XLOG_SEG_SIZE + (offset); \
+       } while (0)
 
 /*
  * Macros for manipulating XLOG pointers
  */
 
-/* Increment an xlogid/segment pair */
-#define NextLogSeg(logId, logSeg)      \
-       do { \
-               if ((logSeg) >= XLogSegsPerFile-1) \
-               { \
-                       (logId)++; \
-                       (logSeg) = 0; \
-               } \
-               else \
-                       (logSeg)++; \
-       } while (0)
-
-/* Decrement an xlogid/segment pair (assume it's not 0,0) */
-#define PrevLogSeg(logId, logSeg)      \
-       do { \
-               if (logSeg) \
-                       (logSeg)--; \
-               else \
-               { \
-                       (logId)--; \
-                       (logSeg) = XLogSegsPerFile-1; \
-               } \
-       } while (0)
-
 /* Align a record pointer to next page */
 #define NextLogPage(recptr) \
        do {    \
                if ((recptr).xrecoff % XLOG_BLCKSZ != 0)        \
-                       (recptr).xrecoff += \
-                               (XLOG_BLCKSZ - (recptr).xrecoff % XLOG_BLCKSZ); \
-               if ((recptr).xrecoff >= XLogFileSize) \
-               {       \
-                       ((recptr).xlogid)++;    \
-                       (recptr).xrecoff = 0; \
-               }       \
+                       XLByteAdvance(recptr, (XLOG_BLCKSZ - (recptr).xrecoff % XLOG_BLCKSZ)); \
        } while (0)
 
 /*
@@ -175,14 +147,11 @@ typedef XLogLongPageHeaderData *XLogLongPageHeader;
  * for example.  (We can assume xrecoff is not zero, since no valid recptr
  * can have that.)
  */
-#define XLByteToSeg(xlrp, logId, logSeg)       \
-       ( logId = (xlrp).xlogid, \
-         logSeg = (xlrp).xrecoff / XLogSegSize \
-       )
-#define XLByteToPrevSeg(xlrp, logId, logSeg)   \
-       ( logId = (xlrp).xlogid, \
-         logSeg = ((xlrp).xrecoff - 1) / XLogSegSize \
-       )
+#define XLByteToSeg(xlrp, logSegNo)    \
+       logSegNo = ((uint64) (xlrp).xlogid * XLogSegmentsPerXLogId) + (xlrp).xrecoff / XLogSegSize
+
+#define XLByteToPrevSeg(xlrp, logSegNo)        \
+       logSegNo = ((uint64) (xlrp).xlogid * XLogSegmentsPerXLogId) + ((xlrp).xrecoff - 1) / XLogSegSize
 
 /*
  * Is an XLogRecPtr within a particular XLOG segment?
@@ -190,13 +159,16 @@ typedef XLogLongPageHeaderData *XLogLongPageHeader;
  * For XLByteInSeg, do the computation at face value.  For XLByteInPrevSeg,
  * a boundary byte is taken to be in the previous segment.
  */
-#define XLByteInSeg(xlrp, logId, logSeg)       \
-       ((xlrp).xlogid == (logId) && \
-        (xlrp).xrecoff / XLogSegSize == (logSeg))
+#define XLByteInSeg(xlrp, logSegNo)    \
+       (((xlrp).xlogid) == (logSegNo) / XLogSegmentsPerXLogId &&                       \
+        ((xlrp).xrecoff / XLogSegSize) == (logSegNo) % XLogSegmentsPerXLogId)
 
-#define XLByteInPrevSeg(xlrp, logId, logSeg)   \
-       ((xlrp).xlogid == (logId) && \
-        ((xlrp).xrecoff - 1) / XLogSegSize == (logSeg))
+#define XLByteInPrevSeg(xlrp, logSegNo)        \
+       (((xlrp).xrecoff == 0) ?                                                                                        \
+               (((xlrp).xlogid - 1) == (logSegNo) / XLogSegmentsPerXLogId && \
+                ((uint32) 0xffffffff) / XLogSegSize == (logSegNo) % XLogSegmentsPerXLogId) : \
+               ((xlrp).xlogid) == (logSegNo) / XLogSegmentsPerXLogId &&        \
+                (((xlrp).xrecoff - 1) / XLogSegSize) == (logSegNo) % XLogSegmentsPerXLogId)
 
 /* Check if an xrecoff value is in a plausible range */
 #define XRecOffIsValid(xrecoff) \
@@ -215,14 +187,23 @@ typedef XLogLongPageHeaderData *XLogLongPageHeader;
  */
 #define MAXFNAMELEN            64
 
-#define XLogFileName(fname, tli, log, seg)     \
-       snprintf(fname, MAXFNAMELEN, "%08X%08X%08X", tli, log, seg)
-
-#define XLogFromFileName(fname, tli, log, seg) \
-       sscanf(fname, "%08X%08X%08X", tli, log, seg)
+#define XLogFileName(fname, tli, logSegNo)     \
+       snprintf(fname, MAXFNAMELEN, "%08X%08X%08X", tli,               \
+                        (uint32) ((logSegNo) / XLogSegmentsPerXLogId), \
+                        (uint32) ((logSegNo) % XLogSegmentsPerXLogId))
+
+#define XLogFromFileName(fname, tli, logSegNo) \
+       do {                                                                                            \
+               uint32 log;                                                                             \
+               uint32 seg;                                                                             \
+               sscanf(fname, "%08X%08X%08X", tli, &log, &seg); \
+               *logSegNo = (uint64) log * XLogSegmentsPerXLogId + seg; \
+       } while (0)
 
-#define XLogFilePath(path, tli, log, seg)      \
-       snprintf(path, MAXPGPATH, XLOGDIR "/%08X%08X%08X", tli, log, seg)
+#define XLogFilePath(path, tli, logSegNo)      \
+       snprintf(path, MAXPGPATH, XLOGDIR "/%08X%08X%08X", tli,                         \
+                        (uint32) ((logSegNo) / XLogSegmentsPerXLogId),                         \
+                        (uint32) ((logSegNo) % XLogSegmentsPerXLogId))
 
 #define TLHistoryFileName(fname, tli)  \
        snprintf(fname, MAXFNAMELEN, "%08X.history", tli)
@@ -233,11 +214,15 @@ typedef XLogLongPageHeaderData *XLogLongPageHeader;
 #define StatusFilePath(path, xlog, suffix)     \
        snprintf(path, MAXPGPATH, XLOGDIR "/archive_status/%s%s", xlog, suffix)
 
-#define BackupHistoryFileName(fname, tli, log, seg, offset) \
-       snprintf(fname, MAXFNAMELEN, "%08X%08X%08X.%08X.backup", tli, log, seg, offset)
+#define BackupHistoryFileName(fname, tli, logSegNo, offset) \
+       snprintf(fname, MAXFNAMELEN, "%08X%08X%08X.%08X.backup", tli, \
+                        (uint32) ((logSegNo) / XLogSegmentsPerXLogId),           \
+                        (uint32) ((logSegNo) % XLogSegmentsPerXLogId), offset)
 
-#define BackupHistoryFilePath(path, tli, log, seg, offset)     \
-       snprintf(path, MAXPGPATH, XLOGDIR "/%08X%08X%08X.%08X.backup", tli, log, seg, offset)
+#define BackupHistoryFilePath(path, tli, logSegNo, offset)     \
+       snprintf(path, MAXPGPATH, XLOGDIR "/%08X%08X%08X.%08X.backup", tli, \
+                        (uint32) ((logSegNo) / XLogSegmentsPerXLogId), \
+                        (uint32) ((logSegNo) % XLogSegmentsPerXLogId), offset)
 
 
 /*
index 5e6d7e600bdd1dcc7a70b52d96e47feef757fb8b..603854884f0eee143001f3c0c195020f19e1de30 100644 (file)
@@ -61,16 +61,16 @@ typedef struct XLogRecPtr
  */
 #define XLByteAdvance(recptr, nbytes)                                          \
        do {                                                                                                    \
-               if (recptr.xrecoff + nbytes >= XLogFileSize)            \
-               {                                                                                                       \
-                       recptr.xlogid += 1;                                                             \
-                       recptr.xrecoff                                                                  \
-                               = recptr.xrecoff + nbytes - XLogFileSize;       \
-               }                                                                                                       \
-               else                                                                                            \
-                       recptr.xrecoff += nbytes;                                               \
+               uint32 oldxrecoff = (recptr).xrecoff;                           \
+               (recptr).xrecoff += nbytes;                                                     \
+               if ((recptr).xrecoff < oldxrecoff)                                      \
+                       (recptr).xlogid += 1;           /* xrecoff wrapped around */    \
        } while (0)
 
+/*
+ * XLogSegNo - physical log file sequence number.
+ */
+typedef uint64 XLogSegNo;
 
 /*
  * TimeLineID (TLI) - identifies different database histories to prevent