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;
/* 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))
/*
* 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;
/*
* 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 */
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);
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);
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);
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);
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;
/* 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;
}
/*
- * 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);
}
XLogRecPtr OldPageRqstPtr;
XLogwrtRqst WriteRqst;
XLogRecPtr NewPageEndPtr;
+ XLogRecPtr NewPageBeginPtr;
XLogPageHeader NewPage;
/*
* 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);
/* 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
/*
* 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;
}
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
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;
}
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;
}
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)));
}
*/
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);
* 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);
}
}
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;
}
else
{
WriteRqstPtr = XLogCtl->xlblocks[Insert->curridx];
+ if (WriteRqstPtr.xrecoff == 0)
+ WriteRqstPtr.xlogid--;
WriteRqstPtr.xrecoff -= freespace;
}
LWLockRelease(WALInsertLock);
{
if (openLogFile >= 0)
{
- if (!XLByteInPrevSeg(LogwrtResult.Write, openLogId, openLogSeg))
+ if (!XLByteInPrevSeg(LogwrtResult.Write, openLogSegNo))
{
XLogFileClose();
}
* 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)
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;
* 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))
{
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");
* 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];
/*
* 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,
/*
* 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");
}
* 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.
LWLockRelease(ControlFileLock);
return false;
}
- NextLogSeg(*log, *seg);
+ (*segno)++;
(*max_advance)--;
- XLogFilePath(path, ThisTimeLineID, *log, *seg);
+ XLogFilePath(path, ThisTimeLineID, *segno);
}
}
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);
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
* 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;
}
* 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];
char path[MAXPGPATH];
int fd;
- XLogFileName(xlogfname, tli, log, seg);
+ XLogFileName(xlogfname, tli, segno);
switch (source)
{
case XLOG_FROM_PG_XLOG:
case XLOG_FROM_STREAM:
- XLogFilePath(path, tli, log, seg);
+ XLogFilePath(path, tli, segno);
restoredFromArchive = false;
break;
bool reload = false;
struct stat statbuf;
- XLogFilePath(xlogfpath, tli, log, seg);
+ XLogFilePath(xlogfpath, tli, segno);
if (stat(xlogfpath, &statbuf) == 0)
{
if (unlink(xlogfpath) != 0)
* 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);
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;
}
* 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;
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");
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;
}
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;
}
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)
*/
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
const char *sp;
int rc;
bool signaled;
- uint32 restartLog;
- uint32 restartSeg;
+ XLogSegNo restartSegNo;
Assert(command && commandName);
* 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);
/*
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++;
* 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);
}
{
/* 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);
}
* 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;
* 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);
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);
* 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,
/* Needn't recheck that slot on future iterations */
if (max_advance > 0)
{
- NextLogSeg(endlogId, endlogSeg);
+ endlogSegNo++;
max_advance--;
}
}
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
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;
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);
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;
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;
{
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)
{
/* 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;
}
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;
}
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;
*/
static void
writeTimeLineHistory(TimeLineID newTLI, TimeLineID parentTLI,
- TimeLineID endTLI, uint32 endLogId, uint32 endLogSeg)
+ TimeLineID endTLI, XLogSegNo endLogSegNo)
{
char path[MAXPGPATH];
char tmppath[MAXPGPATH];
* 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
/* 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;
* Exit archive-recovery state
*/
static void
-exitArchiveRecovery(TimeLineID endTLI, uint32 endLogId, uint32 endLogSeg)
+exitArchiveRecovery(TimeLineID endTLI, XLogSegNo endLogSegNo)
{
char recoveryPath[MAXPGPATH];
char xlogpath[MAXPGPATH];
*/
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);
}
}
* 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);
/*
XLogRecPtr RecPtr,
checkPointLoc,
EndOfLog;
- uint32 endLogId;
- uint32 endLogSeg;
+ XLogSegNo endLogSegNo;
XLogRecord *record;
uint32 freespace;
TransactionId oldestActiveXID;
*/
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
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 */
* 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;
XLogCtlInsert *Insert = &XLogCtl->Insert;
XLogRecData rdata;
uint32 freespace;
- uint32 _logId;
- uint32 _logSeg;
+ XLogSegNo _logSegNo;
TransactionId *inCommitXids;
int nInCommit;
* 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.
* 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);
}
/*
{
XLogRecPtr lastCheckPointRecPtr;
CheckPoint lastCheckPoint;
- uint32 _logId;
- uint32 _logSeg;
+ XLogSegNo _logSegNo;
TimestampTz xtime;
/* use volatile pointer to prevent code rearrangement */
* 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
* 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
* 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;
}
/*
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();
}
* '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)
{
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
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:
}
}
+/*
+ * 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
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;
LWLockRelease(WALInsertLock);
} while (!gotUniqueStartpoint);
- XLByteToSeg(startpoint, _logId, _logSeg);
- XLogFileName(xlogfilename, ThisTimeLineID, _logId, _logSeg);
+ XLByteToSeg(startpoint, _logSegNo);
+ XLogFileName(xlogfilename, ThisTimeLineID, _logSegNo);
/*
* Construct backup label file
char lastxlogfilename[MAXFNAMELEN];
char histfilename[MAXFNAMELEN];
char backupfrom[20];
- uint32 _logId;
- uint32 _logSeg;
+ XLogSegNo _logSegNo;
FILE *lfp;
FILE *fp;
char ch;
*/
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);
/*
* 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)
*/
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;
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;
* 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
*/
if (StandbyMode && bgwriterLaunched)
{
- if (XLogCheckpointNeeded(readId, readSeg))
+ if (XLogCheckpointNeeded(readSegNo))
{
(void) GetRedoRecPtr();
- if (XLogCheckpointNeeded(readId, readSeg))
+ if (XLogCheckpointNeeded(readSegNo))
RequestCheckpoint(CHECKPOINT_CAUSE_XLOG);
}
}
readSource = 0;
}
- XLByteToSeg(*RecPtr, readId, readSeg);
+ XLByteToSeg(*RecPtr, readSegNo);
retry:
/* See if we need to retrieve more data */
if (readFile < 0)
{
readFile =
- XLogFileRead(readId, readSeg, PANIC,
+ XLogFileRead(readSegNo, PANIC,
recoveryTargetTLI,
XLOG_FROM_STREAM, false);
Assert(readFile >= 0);
}
/* 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)
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;
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))
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);