diff -cpr pgsql-orig/src/backend/access/heap/heapam.c pgsql/src/backend/access/heap/heapam.c *** pgsql-orig/src/backend/access/heap/heapam.c 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/backend/access/heap/heapam.c 2006-03-09 10:20:50.000000000 +0900 *************** heap_restrpos(HeapScanDesc scan) *** 2742,2755 **** XLogRecPtr log_heap_clean(Relation reln, Buffer buffer, OffsetNumber *unused, int uncnt) { xl_heap_clean xlrec; XLogRecPtr recptr; XLogRecData rdata[2]; ! /* Caller should not call me on a temp relation */ ! Assert(!reln->rd_istemp); ! ! xlrec.node = reln->rd_node; xlrec.block = BufferGetBlockNumber(buffer); rdata[0].data = (char *) &xlrec; --- 2742,2761 ---- XLogRecPtr log_heap_clean(Relation reln, Buffer buffer, OffsetNumber *unused, int uncnt) { + /* Caller should not call me on a temp relation */ + Assert(!reln->rd_istemp); + + return log_heap_clean_rnode(&reln->rd_node, buffer, unused, uncnt); + } + + XLogRecPtr + log_heap_clean_rnode(RelFileNode *rnode, Buffer buffer, OffsetNumber *unused, int uncnt) + { xl_heap_clean xlrec; XLogRecPtr recptr; XLogRecData rdata[2]; ! xlrec.node = *rnode; xlrec.block = BufferGetBlockNumber(buffer); rdata[0].data = (char *) &xlrec; diff -cpr pgsql-orig/src/backend/access/transam/xlog.c pgsql/src/backend/access/transam/xlog.c *** pgsql-orig/src/backend/access/transam/xlog.c 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/backend/access/transam/xlog.c 2006-03-09 10:20:50.000000000 +0900 *************** CreateCheckPoint(bool shutdown, bool for *** 5215,5221 **** CheckPointCLOG(); CheckPointSUBTRANS(); CheckPointMultiXact(); ! FlushBufferPool(); /* We deliberately delay 2PC checkpointing as long as possible */ CheckPointTwoPhase(checkPoint.redo); --- 5215,5221 ---- CheckPointCLOG(); CheckPointSUBTRANS(); CheckPointMultiXact(); ! FlushBufferPool(shutdown); /* We deliberately delay 2PC checkpointing as long as possible */ CheckPointTwoPhase(checkPoint.redo); diff -cpr pgsql-orig/src/backend/commands/dbcommands.c pgsql/src/backend/commands/dbcommands.c *** pgsql-orig/src/backend/commands/dbcommands.c 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/backend/commands/dbcommands.c 2006-03-09 10:20:50.000000000 +0900 *************** createdb(const CreatedbStmt *stmt) *** 360,366 **** * up-to-date for the copy. (We really only need to flush buffers for the * source database, but bufmgr.c provides no API for that.) */ ! BufferSync(); /* * Once we start copying subdirectories, we need to be able to clean 'em --- 360,366 ---- * up-to-date for the copy. (We really only need to flush buffers for the * source database, but bufmgr.c provides no API for that.) */ ! BufferSync(false); /* * Once we start copying subdirectories, we need to be able to clean 'em *************** dbase_redo(XLogRecPtr lsn, XLogRecord *r *** 1361,1367 **** * up-to-date for the copy. (We really only need to flush buffers for * the source database, but bufmgr.c provides no API for that.) */ ! BufferSync(); /* * Copy this subdirectory to the new location --- 1361,1367 ---- * up-to-date for the copy. (We really only need to flush buffers for * the source database, but bufmgr.c provides no API for that.) */ ! BufferSync(false); /* * Copy this subdirectory to the new location diff -cpr pgsql-orig/src/backend/storage/buffer/bufmgr.c pgsql/src/backend/storage/buffer/bufmgr.c *** pgsql-orig/src/backend/storage/buffer/bufmgr.c 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/backend/storage/buffer/bufmgr.c 2006-03-09 10:22:41.000000000 +0900 *************** *** 53,58 **** --- 53,77 ---- #include "utils/resowner.h" #include "pgstat.h" + #include "access/heapam.h" + #include "storage/procarray.h" + #include "storage/freespace.h" + #include "utils/tqual.h" + + + /*#define BGVACUUM_DEBUG*/ + #define BGVACUUM_STAT + + #ifdef BGVACUUM_STAT + int BgVacuumStat_Vacuum = 0; + int BgVacuumStat_Useless = 0; + int BgVacuumStat_Conflict = 0; + int BgVacuumStat_Skip = 0; + int BgVacuumStat_Unknown = 0; + #define BGVACUUM_STAT_INC(name) (++(name)) + #else /* BGVACUUM_STAT */ + #define BGVACUUM_STAT_INC(name) ((void)0) + #endif /* BGVACUUM_STAT */ /* Note: these two macros only work on shared buffers, not local ones! */ #define BufHdrGetBlock(bufHdr) ((Block) (BufferBlocks + ((Size) (bufHdr)->buf_id) * BLCKSZ)) *************** double bgwriter_all_percent = 0.333; *** 73,78 **** --- 92,101 ---- int bgwriter_lru_maxpages = 5; int bgwriter_all_maxpages = 5; + bool bgvacuum_autotune = true; + bool bgvacuum_fsm = true; + bool bgvacuum_relation = true; + long NDirectFileRead; /* some I/O's are direct file access. bypass * bufmgr */ *************** static bool IsForInput; *** 86,97 **** /* local state for LockBufferForCleanup */ static volatile BufferDesc *PinCountWaitBuf = NULL; static bool PinBuffer(volatile BufferDesc *buf); static void PinBuffer_Locked(volatile BufferDesc *buf); static void UnpinBuffer(volatile BufferDesc *buf, bool fixOwner, bool normalAccess); ! static bool SyncOneBuffer(int buf_id, bool skip_pinned); static void WaitIO(volatile BufferDesc *buf); static bool StartBufferIO(volatile BufferDesc *buf, bool forInput); static void TerminateBufferIO(volatile BufferDesc *buf, bool clear_dirty, --- 109,131 ---- /* local state for LockBufferForCleanup */ static volatile BufferDesc *PinCountWaitBuf = NULL; + typedef enum SyncMode + { + SyncLru, /* by bgwriter 'lru' writes. */ + SyncAll, /* by bgwriter 'all' writes. */ + SyncFlush, /* by checkpoint. */ + SyncShutdown, /* by shutdown checkpoint. */ + } SyncMode; static bool PinBuffer(volatile BufferDesc *buf); static void PinBuffer_Locked(volatile BufferDesc *buf); static void UnpinBuffer(volatile BufferDesc *buf, bool fixOwner, bool normalAccess); ! static bool SyncOneBuffer(int buf_id, SyncMode mode, TransactionId xmin); ! static bool RelationVacuumPage(volatile BufferDesc *bufHdr, ! SyncMode mode, TransactionId xmin); ! static bool RelationDefragPage(BufferDesc *bufHdr, TransactionId xmin); ! static void RelationRecordFreeSpace(BufferDesc *bufHdr); static void WaitIO(volatile BufferDesc *buf); static bool StartBufferIO(volatile BufferDesc *buf, bool forInput); static void TerminateBufferIO(volatile BufferDesc *buf, bool clear_dirty, *************** BufferAlloc(Relation reln, *** 507,514 **** oldTag = buf->tag; oldFlags = buf->flags; buf->tag = newTag; ! buf->flags &= ~(BM_VALID | BM_DIRTY | BM_JUST_DIRTIED | BM_IO_ERROR); buf->flags |= BM_TAG_VALID; buf->usage_count = 0; UnlockBufHdr(buf); --- 541,553 ---- oldTag = buf->tag; oldFlags = buf->flags; buf->tag = newTag; ! buf->flags &= ~(BM_VALID | BM_DIRTY | BM_JUST_DIRTIED | BM_IO_ERROR | BM_RELATION); buf->flags |= BM_TAG_VALID; + if (!reln->rd_istemp && reln->rd_rel && + reln->rd_rel->relkind == RELKIND_RELATION) + { + buf->flags |= BM_RELATION; + } buf->usage_count = 0; UnlockBufHdr(buf); *************** UnpinBuffer(volatile BufferDesc *buf, bo *** 892,907 **** * This is called at checkpoint time to write out all dirty shared buffers. */ void ! BufferSync(void) { int buf_id; int num_to_scan; int absorb_counter; /* * Find out where to start the circular scan. */ ! buf_id = StrategySyncStart(); /* Make sure we can handle the pin inside SyncOneBuffer */ ResourceOwnerEnlargeBuffers(CurrentResourceOwner); --- 931,954 ---- * This is called at checkpoint time to write out all dirty shared buffers. */ void ! BufferSync(bool shutdown) { int buf_id; int num_to_scan; int absorb_counter; + int numGetBuffer; + TransactionId xmin; + SyncMode mode = (shutdown ? SyncShutdown : SyncFlush); + + if (bgvacuum_relation) + xmin = GetOldestXmin(true); + else + xmin = InvalidTransactionId; /* * Find out where to start the circular scan. */ ! buf_id = StrategySyncStart(&numGetBuffer); /* Make sure we can handle the pin inside SyncOneBuffer */ ResourceOwnerEnlargeBuffers(CurrentResourceOwner); *************** BufferSync(void) *** 913,919 **** absorb_counter = WRITES_PER_ABSORB; while (num_to_scan-- > 0) { ! if (SyncOneBuffer(buf_id, false)) { /* * If in bgwriter, absorb pending fsync requests after each --- 960,966 ---- absorb_counter = WRITES_PER_ABSORB; while (num_to_scan-- > 0) { ! if (SyncOneBuffer(buf_id, mode, xmin)) { /* * If in bgwriter, absorb pending fsync requests after each *************** BgBufferSync(void) *** 944,949 **** --- 991,1003 ---- int buf_id2; int num_to_scan; int num_written; + int numGetBuffer; + TransactionId xmin; + + if (bgvacuum_relation) + xmin = GetOldestXmin(true); + else + xmin = InvalidTransactionId; /* Make sure we can handle the pin inside SyncOneBuffer */ ResourceOwnerEnlargeBuffers(CurrentResourceOwner); *************** BgBufferSync(void) *** 975,981 **** { if (++buf_id1 >= NBuffers) buf_id1 = 0; ! if (SyncOneBuffer(buf_id1, false)) { if (++num_written >= bgwriter_all_maxpages) break; --- 1029,1035 ---- { if (++buf_id1 >= NBuffers) buf_id1 = 0; ! if (SyncOneBuffer(buf_id1, SyncAll, xmin)) { if (++num_written >= bgwriter_all_maxpages) break; *************** BgBufferSync(void) *** 987,1002 **** * This loop considers only unpinned buffers close to the clock sweep * point. */ ! if (bgwriter_lru_percent > 0.0 && bgwriter_lru_maxpages > 0) { num_to_scan = (int) ((NBuffers * bgwriter_lru_percent + 99) / 100); num_written = 0; ! buf_id2 = StrategySyncStart(); while (num_to_scan-- > 0) { ! if (SyncOneBuffer(buf_id2, true)) { if (++num_written >= bgwriter_lru_maxpages) break; --- 1041,1066 ---- * This loop considers only unpinned buffers close to the clock sweep * point. */ ! if (bgwriter_lru_percent > 0.0) { num_to_scan = (int) ((NBuffers * bgwriter_lru_percent + 99) / 100); num_written = 0; ! buf_id2 = StrategySyncStart(&numGetBuffer); ! if (bgvacuum_autotune) ! { ! if (bgwriter_lru_maxpages < numGetBuffer) ! bgwriter_lru_maxpages += 1; ! else if(bgwriter_lru_maxpages > numGetBuffer) ! bgwriter_lru_maxpages -= 1; ! ! /* starts writing a little ahead. */ ! buf_id2 = (buf_id2 + bgwriter_lru_maxpages) % NBuffers; ! } while (num_to_scan-- > 0) { ! if (SyncOneBuffer(buf_id2, SyncLru, xmin)) { if (++num_written >= bgwriter_lru_maxpages) break; *************** BgBufferSync(void) *** 1020,1027 **** * Note: caller must have done ResourceOwnerEnlargeBuffers. */ static bool ! SyncOneBuffer(int buf_id, bool skip_pinned) { volatile BufferDesc *bufHdr = &BufferDescriptors[buf_id]; /* --- 1084,1093 ---- * Note: caller must have done ResourceOwnerEnlargeBuffers. */ static bool ! SyncOneBuffer(int buf_id, SyncMode mode, TransactionId xmin) { + BufFlags flags; + bool noref; volatile BufferDesc *bufHdr = &BufferDescriptors[buf_id]; /* *************** SyncOneBuffer(int buf_id, bool skip_pinn *** 1034,1049 **** * upcoming changes and so we are not required to write such dirty buffer. */ LockBufHdr(bufHdr); ! if (!(bufHdr->flags & BM_VALID) || !(bufHdr->flags & BM_DIRTY)) { UnlockBufHdr(bufHdr); return false; } ! if (skip_pinned && ! (bufHdr->refcount != 0 || bufHdr->usage_count != 0)) { UnlockBufHdr(bufHdr); ! return false; } /* --- 1100,1118 ---- * upcoming changes and so we are not required to write such dirty buffer. */ LockBufHdr(bufHdr); ! ! noref = (bufHdr->refcount == 0); ! if (mode == SyncLru && (!noref || bufHdr->usage_count != 0)) { UnlockBufHdr(bufHdr); return false; } ! ! flags = bufHdr->flags; ! if (!(flags & BM_VALID) || !(flags & BM_DIRTY)) { UnlockBufHdr(bufHdr); ! return mode == SyncLru; } /* *************** SyncOneBuffer(int buf_id, bool skip_pinn *** 1051,1057 **** * buffer is clean by the time we've locked it.) */ PinBuffer_Locked(bufHdr); ! LWLockAcquire(bufHdr->content_lock, LW_SHARED); FlushBuffer(bufHdr, NULL); --- 1120,1145 ---- * buffer is clean by the time we've locked it.) */ PinBuffer_Locked(bufHdr); ! ! if (flags & BM_RELATION) ! { ! if (bgvacuum_relation && mode != SyncShutdown && noref) ! { ! if (!RelationVacuumPage(bufHdr, mode, xmin)) ! return false; ! } ! else ! { ! BGVACUUM_STAT_INC( BgVacuumStat_Skip ); ! LWLockAcquire(bufHdr->content_lock, LW_SHARED); ! } ! } ! else ! { ! /* none-relation page */ ! BGVACUUM_STAT_INC( BgVacuumStat_Unknown ); ! LWLockAcquire(bufHdr->content_lock, LW_SHARED); ! } FlushBuffer(bufHdr, NULL); *************** PrintBufferLeakWarning(Buffer buffer) *** 1216,1221 **** --- 1304,1328 ---- buf->refcount, loccount); } + #ifdef BGVACUUM_STAT + static void + PrintBgVacuumStat(const char* name) + { + elog(LOG, "BGVACUUM(%s) :\t%d\t%d\t%d\t%d\t%d", + name, + BgVacuumStat_Vacuum, + BgVacuumStat_Useless, + BgVacuumStat_Conflict, + BgVacuumStat_Skip, + BgVacuumStat_Unknown); + BgVacuumStat_Vacuum = 0; + BgVacuumStat_Useless = 0; + BgVacuumStat_Conflict = 0; + BgVacuumStat_Skip = 0; + BgVacuumStat_Unknown = 0; + } + #endif + /* * FlushBufferPool * *************** PrintBufferLeakWarning(Buffer buffer) *** 1224,1232 **** * flushed. */ void ! FlushBufferPool(void) { ! BufferSync(); smgrsync(); } --- 1331,1349 ---- * flushed. */ void ! FlushBufferPool(bool shutdown) { ! #ifdef BGVACUUM_STAT ! elog(LOG, "AUTOLRU :\t%d", bgwriter_lru_maxpages); ! PrintBgVacuumStat("background"); ! #endif ! ! BufferSync(shutdown); ! ! #ifdef BGVACUUM_STAT ! PrintBgVacuumStat("checkpoint"); ! #endif ! smgrsync(); } *************** buffer_write_error_callback(void *arg) *** 2118,2120 **** --- 2235,2405 ---- bufHdr->tag.rnode.dbNode, bufHdr->tag.rnode.relNode); } + + /* + * RelationVacuumPage -- vacuum one relation page + * + * @param bufHdr Must be pinned. + * @param mode Sync mode. + * @param xmin Vacuum threshould. + * @result Should write? If true, bufHdr is shared locked. + */ + static bool + RelationVacuumPage(volatile BufferDesc *bufHdr, + SyncMode mode, TransactionId xmin) + { + BufferDesc *buf; + bool hasSuperLock; + + /* We can vacuum the page only if the super exclusive lock is held. */ + + if (!LWLockConditionalAcquire(bufHdr->content_lock, LW_EXCLUSIVE)) + { + BGVACUUM_STAT_INC( BgVacuumStat_Conflict ); + LWLockAcquire(bufHdr->content_lock, LW_SHARED); + return true; + } + + LockBufHdr(bufHdr); + hasSuperLock = (bufHdr->refcount == 1); /* pinned only by me */ + UnlockBufHdr(bufHdr); + + if (!hasSuperLock) + { + LWLockDowngrade(bufHdr->content_lock); + BGVACUUM_STAT_INC( BgVacuumStat_Conflict ); + return true; + } + + buf = /* volatile_cast */ (BufferDesc *) bufHdr; + + if (!RelationDefragPage(buf, xmin)) + { + /* no dead tuples */ + LWLockDowngrade(bufHdr->content_lock); + BGVACUUM_STAT_INC( BgVacuumStat_Useless ); + return true; + } + + /* vacuum was performed */ + BGVACUUM_STAT_INC( BgVacuumStat_Vacuum ); + switch (mode) + { + case SyncLru: + case SyncAll: + /* record free space and skip this write */ + LWLockRelease(bufHdr->content_lock); + RelationRecordFreeSpace(buf); /* XXX: should call after release pin? */ + UnpinBuffer(bufHdr, true, true /* prevent reuse for a while */); + return false; + case SyncFlush: + /* record free space and flush */ + LWLockDowngrade(bufHdr->content_lock); + RelationRecordFreeSpace(buf); /* XXX: should call after release lock and pin? */ + return true; + default: + elog(ERROR, "unexpected SyncMode %d", mode); + return false; /* keep compiler quiet */ + } + } + + /* + * RelationDefragPage -- vacuum one relation page + * + * @result Vacuum performed? + */ + static bool + RelationDefragPage(BufferDesc *bufHdr, TransactionId xmin) + { + Buffer buffer = BufferDescriptorGetBuffer(bufHdr); + Page page = (Page) BufHdrGetBlock(bufHdr); + OffsetNumber offnum, maxoff; + int num_kill = 0; + #ifdef BGVACUUM_DEBUG + int num_alive = 0; + int num_dead = 0; + #endif + OffsetNumber unused[MaxOffsetNumber]; + int uncnt; + XLogRecPtr recptr; + + if (PageIsNew(page) || PageIsEmpty(page)) + return false; + + maxoff = PageGetMaxOffsetNumber(page); + for (offnum = FirstOffsetNumber; + offnum <= maxoff; + offnum = OffsetNumberNext(offnum)) + { + ItemId itemid = PageGetItemId(page, offnum); + HeapTupleHeader tuple; + + if (!ItemIdIsUsed(itemid)) + continue; + + tuple = (HeapTupleHeader) PageGetItem(page, itemid); + if (HeapTupleHeaderGetXmax(tuple) == FrozenTransactionId) + { /* This tuple has been truncated already. */ + #ifdef BGVACUUM_DEBUG + ++num_dead; + #endif + continue; + } + switch (HeapTupleSatisfiesVacuum(tuple, xmin, buffer)) + { + /* + * We can truncate recently dead tuples, because ctid chains in the headers + * still remain. + */ + case HEAPTUPLE_DEAD: + case HEAPTUPLE_RECENTLY_DEAD: + Assert(!(tuple->t_infomask & HEAP_IS_LOCKED)); + tuple->t_natts = 0; + tuple->t_infomask &= + ~(HEAP_HASNULL | HEAP_HASVARWIDTH | HEAP_HASEXTENDED | HEAP_HASOID); + HeapTupleHeaderSetXmin(tuple, FrozenTransactionId); + HeapTupleHeaderSetXmax(tuple, FrozenTransactionId); + itemid->lp_len = MAXALIGN(offsetof(HeapTupleHeaderData, t_bits)); + ++num_kill; + break; + case HEAPTUPLE_LIVE: + case HEAPTUPLE_INSERT_IN_PROGRESS: + case HEAPTUPLE_DELETE_IN_PROGRESS: + #ifdef BGVACUUM_DEBUG + ++num_alive; + #endif + break; + default: + elog(ERROR, "unexpected HeapTupleSatisfiesVacuum result"); + } + } + + #ifdef BGVACUUM_DEBUG + elog(LOG, "BGVACUUM TABLE: ID=%d, alive=%d, kill=%d, dead=%d", + buffer, num_alive, num_kill, num_dead); + #endif + + if (num_kill <= 0) + return false; + + /* TODO: XLOG stuff for truncation. */ + + uncnt = PageRepairFragmentation(page, unused); + + /* XLOG stuff for defragmentation. */ + recptr = log_heap_clean_rnode(&bufHdr->tag.rnode, buffer, unused, uncnt); + PageSetLSN(page, recptr); + PageSetTLI(page, ThisTimeLineID); + + return true; + } + + static void + RelationRecordFreeSpace(BufferDesc *bufHdr) + { + if (bgvacuum_fsm) + { + Size space = PageGetFreeSpace((Page) BufHdrGetBlock(bufHdr)); + RecordFreeSpace(&bufHdr->tag.rnode, bufHdr->tag.blockNum, space); + } + } diff -cpr pgsql-orig/src/backend/storage/buffer/freelist.c pgsql/src/backend/storage/buffer/freelist.c *** pgsql-orig/src/backend/storage/buffer/freelist.c 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/backend/storage/buffer/freelist.c 2006-03-09 10:20:50.000000000 +0900 *************** typedef struct *** 27,32 **** --- 27,34 ---- /* Clock sweep hand: index of next buffer to consider grabbing */ int nextVictimBuffer; + int numGetBuffer; /* Count of buffer requests */ + int firstFreeBuffer; /* Head of list of unused buffers */ int lastFreeBuffer; /* Tail of list of unused buffers */ *************** StrategyGetBuffer(void) *** 63,68 **** --- 65,72 ---- LWLockAcquire(BufFreelistLock, LW_EXCLUSIVE); + StrategyControl->numGetBuffer++; + /* * Try to get a buffer from the freelist. Note that the freeNext fields * are considered to be protected by the BufFreelistLock not the *************** StrategyFreeBuffer(volatile BufferDesc * *** 176,182 **** * BufferSync() will proceed circularly around the buffer array from there. */ int ! StrategySyncStart(void) { int result; --- 180,186 ---- * BufferSync() will proceed circularly around the buffer array from there. */ int ! StrategySyncStart(int *numGetBuffer) { int result; *************** StrategySyncStart(void) *** 186,191 **** --- 190,197 ---- */ LWLockAcquire(BufFreelistLock, LW_EXCLUSIVE); result = StrategyControl->nextVictimBuffer; + *numGetBuffer = StrategyControl->numGetBuffer; + StrategyControl->numGetBuffer = 0; LWLockRelease(BufFreelistLock); return result; } *************** StrategyInitialize(bool init) *** 263,268 **** --- 269,276 ---- /* Initialize the clock sweep pointer */ StrategyControl->nextVictimBuffer = 0; + + StrategyControl->numGetBuffer = 0; } else Assert(!init); diff -cpr pgsql-orig/src/backend/storage/freespace/freespace.c pgsql/src/backend/storage/freespace/freespace.c *** pgsql-orig/src/backend/storage/freespace/freespace.c 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/backend/storage/freespace/freespace.c 2006-03-09 10:20:50.000000000 +0900 *************** *** 70,75 **** --- 70,76 ---- #include "storage/lwlock.h" #include "storage/shmem.h" + extern bool bgvacuum_fsm; /*---------- * During database shutdown, we store the contents of FSM into a disk file, *************** RecordAndGetPageWithFreeSpace(RelFileNod *** 339,344 **** --- 340,399 ---- } /* + * RecordFreeSpace - update info about a page. + */ + void + RecordFreeSpace(RelFileNode *rel, + BlockNumber page, + Size avail) + { + FSMRelation *fsmrel; + + LWLockAcquire(FreeSpaceLock, LW_EXCLUSIVE); + + fsmrel = create_fsm_rel(rel); + if (fsmrel->storedPages > 0) + { + /* + elog(LOG, "RecordFreeSpace(ID=%d, stored=%d, avail=%d/%d)", + rel->relNode, fsmrel->storedPages, avail, fsmrel->avgRequest); + */ + if (avail >= fsmrel->avgRequest) + fsm_record_free_space(fsmrel, page, avail); + } + else + { + int i; + int nPages; + int curAlloc; + int curAllocPages; + FSMPageData *newLocation; + + nPages = MaxFSMPages / MaxFSMRelations; + curAlloc = realloc_fsm_rel(fsmrel, nPages, false); + curAllocPages = curAlloc * CHUNKPAGES; + + elog(LOG, "RecordFreeSpace(NEW: ID=%d, stored=%d, avail=%d)", + rel->relNode, curAllocPages, avail); + + newLocation = (FSMPageData *) + (FreeSpaceMap->arena + fsmrel->firstChunk * CHUNKBYTES); + + FSMPageSetPageNum(newLocation, page); + FSMPageSetSpace(newLocation, avail); + newLocation++; + for (i = 1; i < curAllocPages; i++) + { + FSMPageSetPageNum(newLocation, InvalidBlockNumber); + FSMPageSetSpace(newLocation, 0); + newLocation++; + } + fsmrel->storedPages = curAllocPages; + } + LWLockRelease(FreeSpaceLock); + } + + /* * GetAvgFSMRequestSize - get average FSM request size for a relation. * * If the relation is not known to FSM, return a default value. *************** RecordRelationFreeSpace(RelFileNode *rel *** 420,426 **** FSMPageSetSpace(newLocation, avail); newLocation++; } ! fsmrel->storedPages = nPages; } else { --- 475,487 ---- FSMPageSetSpace(newLocation, avail); newLocation++; } ! for (; i < curAllocPages; i++) ! { ! FSMPageSetPageNum(newLocation, InvalidBlockNumber); ! FSMPageSetSpace(newLocation, 0); ! newLocation++; ! } ! fsmrel->storedPages = curAllocPages; } else { *************** fsm_record_free_space(FSMRelation *fsmre *** 1218,1223 **** --- 1279,1345 ---- info += pageIndex; FSMPageSetSpace(info, spaceAvail); } + else if (bgvacuum_fsm && spaceAvail >= fsmrel->avgRequest) + { + int i, begin, end, victim = -1; + Size minAvail = spaceAvail; + FSMPageData *info; + int victim_search_range = 10; + + info = (FSMPageData *) + (FreeSpaceMap->arena + fsmrel->firstChunk * CHUNKBYTES); + + begin = Max(0, pageIndex - victim_search_range); + end = Min(pageIndex + victim_search_range, fsmrel->storedPages); + + for (i = begin; i < end; i++) + { + Size avail = FSMPageGetSpace(info + i); + if (avail < minAvail) + { + victim = i; + minAvail = avail; + } + } + if (victim >= 0) + { + #ifdef FREESPACE_DEBUG + BlockNumber victimPage = FSMPageGetPageNum(info+victim); + #endif + if (victim < pageIndex) + { + memmove(info + victim, info + victim + 1, (pageIndex - victim) * sizeof(FSMPageData)); + info += pageIndex - 1; + } + else + { + memmove(info + pageIndex + 1, info + pageIndex, (victim - pageIndex) * sizeof(FSMPageData)); + info += pageIndex; + } + FSMPageSetPageNum(info, page); + FSMPageSetSpace(info, spaceAvail); + + #ifdef FREESPACE_DEBUG + { + FSMPageData *p = (FSMPageData *) + (FreeSpaceMap->arena + fsmrel->firstChunk * CHUNKBYTES); + for (i = 0; i < fsmrel->storedPages-1; i++) + { + if(FSMPageGetPageNum(p+i) > FSMPageGetPageNum(p+i+1)) + { + elog(WARNING, "INSERTED page=%u, index=%u", page, pageIndex); + elog(WARNING, "VICTIM page=%u, index=%u, avail=%u", victimPage, victim, minAvail); + for (i = 0; i < fsmrel->storedPages; i++) + { + elog(WARNING, "[%5d] %5u = %5d", i, FSMPageGetPageNum(p+i), FSMPageGetSpace(p+i)); + } + elog(FATAL, "FSM corrupted (not sorted)"); + } + } + } + #endif + } + } else { /* diff -cpr pgsql-orig/src/backend/storage/lmgr/lwlock.c pgsql/src/backend/storage/lmgr/lwlock.c *** pgsql-orig/src/backend/storage/lmgr/lwlock.c 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/backend/storage/lmgr/lwlock.c 2006-03-09 10:20:50.000000000 +0900 *************** LWLockHeldByMe(LWLockId lockid) *** 604,606 **** --- 604,662 ---- } return false; } + + /* + * LWLockDowngrade - downgrade exclusive lock to shared lock + */ + void + LWLockDowngrade(LWLockId lockid) + { + volatile LWLock *lock = &(LWLockArray[lockid].lock); + PGPROC *head; + PGPROC *proc; + + /* TODO: Check lockid is exclusive locked by me */ + PRINT_LWDEBUG("LWLockDowngrade", lockid, lock); + + /* Acquire mutex. Time spent holding mutex should be short! */ + SpinLockAcquire(&lock->mutex); + + Assert (lock->exclusive > 0); + lock->exclusive--; + lock->shared++; + + head = lock->head; + if (head != NULL) + { + if (lock->releaseOK && !head->lwExclusive) + { + proc = head; + while (proc->lwWaitLink != NULL && + !proc->lwWaitLink->lwExclusive) + proc = proc->lwWaitLink; + lock->head = proc->lwWaitLink; + proc->lwWaitLink = NULL; + lock->releaseOK = false; + } + else + { + /* lock is still held, can't awaken anything */ + head = NULL; + } + } + + SpinLockRelease(&lock->mutex); + + /* + * Awaken any waiters I removed from the queue. + */ + while (head != NULL) + { + LOG_LWDEBUG("LWLockDowngrade", lockid, "release waiter"); + proc = head; + head = proc->lwWaitLink; + proc->lwWaitLink = NULL; + proc->lwWaiting = false; + PGSemaphoreUnlock(&proc->sem); + } + } diff -cpr pgsql-orig/src/backend/utils/misc/guc.c pgsql/src/backend/utils/misc/guc.c *** pgsql-orig/src/backend/utils/misc/guc.c 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/backend/utils/misc/guc.c 2006-03-09 10:20:50.000000000 +0900 *************** static struct config_bool ConfigureNames *** 1003,1008 **** --- 1003,1035 ---- false, NULL, NULL }, + { + {"bgvacuum_autotune", PGC_SIGHUP, RESOURCES, + gettext_noop("Enables the bgwriter's auto tuning of bgwriter_lru_maxpages."), + NULL + }, + &bgvacuum_autotune, + true, NULL, NULL + }, + + { + {"bgvacuum_fsm", PGC_SIGHUP, RESOURCES, + gettext_noop("Enables the auto maintenance of free space map."), + NULL + }, + &bgvacuum_fsm, + true, NULL, NULL + }, + + { + {"bgvacuum_relation", PGC_SIGHUP, RESOURCES, + gettext_noop("Enables the bgwriter's vacuum of relations."), + NULL + }, + &bgvacuum_relation, + true, NULL, NULL + }, + /* End-of-list marker */ { {NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL diff -cpr pgsql-orig/src/include/access/heapam.h pgsql/src/include/access/heapam.h *** pgsql-orig/src/include/access/heapam.h 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/include/access/heapam.h 2006-03-09 10:20:50.000000000 +0900 *************** extern void heap_redo(XLogRecPtr lsn, XL *** 181,186 **** --- 181,188 ---- extern void heap_desc(char *buf, uint8 xl_info, char *rec); extern XLogRecPtr log_heap_clean(Relation reln, Buffer buffer, OffsetNumber *unused, int uncnt); + extern XLogRecPtr log_heap_clean_rnode(RelFileNode *rnode, Buffer buffer, + OffsetNumber *unused, int uncnt); extern XLogRecPtr log_heap_move(Relation reln, Buffer oldbuf, ItemPointerData from, Buffer newbuf, HeapTuple newtup); diff -cpr pgsql-orig/src/include/storage/buf_internals.h pgsql/src/include/storage/buf_internals.h *** pgsql-orig/src/include/storage/buf_internals.h 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/include/storage/buf_internals.h 2006-03-09 10:20:50.000000000 +0900 *************** *** 35,40 **** --- 35,41 ---- #define BM_IO_ERROR (1 << 4) /* previous I/O failed */ #define BM_JUST_DIRTIED (1 << 5) /* dirtied since write started */ #define BM_PIN_COUNT_WAITER (1 << 6) /* have waiter for sole pin */ + #define BM_RELATION (1 << 7) /* relation page */ typedef bits16 BufFlags; *************** extern long int LocalBufferFlushCount; *** 175,181 **** /* freelist.c */ extern volatile BufferDesc *StrategyGetBuffer(void); extern void StrategyFreeBuffer(volatile BufferDesc *buf, bool at_head); ! extern int StrategySyncStart(void); extern Size StrategyShmemSize(void); extern void StrategyInitialize(bool init); --- 176,182 ---- /* freelist.c */ extern volatile BufferDesc *StrategyGetBuffer(void); extern void StrategyFreeBuffer(volatile BufferDesc *buf, bool at_head); ! extern int StrategySyncStart(int *numGetBuffer); extern Size StrategyShmemSize(void); extern void StrategyInitialize(bool init); diff -cpr pgsql-orig/src/include/storage/bufmgr.h pgsql/src/include/storage/bufmgr.h *** pgsql-orig/src/include/storage/bufmgr.h 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/include/storage/bufmgr.h 2006-03-09 10:20:50.000000000 +0900 *************** extern double bgwriter_all_percent; *** 32,37 **** --- 32,41 ---- extern int bgwriter_lru_maxpages; extern int bgwriter_all_maxpages; + extern bool bgvacuum_autotune; + extern bool bgvacuum_fsm; + extern bool bgvacuum_relation; + /* in buf_init.c */ extern DLLIMPORT char *BufferBlocks; extern DLLIMPORT int32 *PrivateRefCount; *************** extern char *ShowBufferUsage(void); *** 128,134 **** extern void ResetBufferUsage(void); extern void AtEOXact_Buffers(bool isCommit); extern void PrintBufferLeakWarning(Buffer buffer); ! extern void FlushBufferPool(void); extern BlockNumber BufferGetBlockNumber(Buffer buffer); extern BlockNumber RelationGetNumberOfBlocks(Relation relation); extern void RelationTruncate(Relation rel, BlockNumber nblocks); --- 132,138 ---- extern void ResetBufferUsage(void); extern void AtEOXact_Buffers(bool isCommit); extern void PrintBufferLeakWarning(Buffer buffer); ! extern void FlushBufferPool(bool shutdown); extern BlockNumber BufferGetBlockNumber(Buffer buffer); extern BlockNumber RelationGetNumberOfBlocks(Relation relation); extern void RelationTruncate(Relation rel, BlockNumber nblocks); *************** extern void LockBufferForCleanup(Buffer *** 153,159 **** extern void AbortBufferIO(void); extern void BufmgrCommit(void); ! extern void BufferSync(void); extern void BgBufferSync(void); extern void AtProcExit_LocalBuffers(void); --- 157,163 ---- extern void AbortBufferIO(void); extern void BufmgrCommit(void); ! extern void BufferSync(bool shutdown); extern void BgBufferSync(void); extern void AtProcExit_LocalBuffers(void); diff -cpr pgsql-orig/src/include/storage/freespace.h pgsql/src/include/storage/freespace.h *** pgsql-orig/src/include/storage/freespace.h 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/include/storage/freespace.h 2006-03-09 10:20:50.000000000 +0900 *************** extern BlockNumber RecordAndGetPageWithF *** 140,145 **** --- 140,148 ---- BlockNumber oldPage, Size oldSpaceAvail, Size spaceNeeded); + extern void RecordFreeSpace(RelFileNode *rel, + BlockNumber page, + Size avail); extern Size GetAvgFSMRequestSize(RelFileNode *rel); extern void RecordRelationFreeSpace(RelFileNode *rel, int nPages, diff -cpr pgsql-orig/src/include/storage/lwlock.h pgsql/src/include/storage/lwlock.h *** pgsql-orig/src/include/storage/lwlock.h 2006-03-09 10:19:20.000000000 +0900 --- pgsql/src/include/storage/lwlock.h 2006-03-09 10:20:50.000000000 +0900 *************** extern bool LWLockConditionalAcquire(LWL *** 70,75 **** --- 70,76 ---- extern void LWLockRelease(LWLockId lockid); extern void LWLockReleaseAll(void); extern bool LWLockHeldByMe(LWLockId lockid); + extern void LWLockDowngrade(LWLockId lockid); extern int NumLWLocks(void); extern Size LWLockShmemSize(void);