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