From ebaf6feaf0530fb0eace516bb1c8487b5ef9fa7f Mon Sep 17 00:00:00 2001 From: Shawn Debnath Date: Wed, 27 Feb 2019 18:58:58 +0000 Subject: [PATCH] Refactor the fsync mechanism to support future SMGR implementations. In anticipation of proposed block storage managers alongside md.c that map bufmgr.c blocks to files optimised for different usage patterns: 1. Move the system for requesting and processing fsyncs out of md.c into storage/sync/sync.c with definitions in include/storage/sync.h. ProcessSyncRequests() is now responsible for processing the sync requests during checkpoint. 2. Removed the need for specific storage managers to implement pre and post checkpoint callbacks. These are now executed by the sync mechanism. 3. We now embed the fork number and the segment number as part of the hash key for the pending ops table. This eliminates the bitmapset based segment tracking for each relfilenode during fsync as not all storage managers may map their segments from zero. 4. Each sync request now must include a type: sync, forget, forget hierarchy, or unlink, and the owner who will be responsible for generating paths or matching forget requests. 5. For cancelling relation sync requests, we now must send a forget request for each fork and segment in the relation. 6. We do not rely on smgr to provide the file descriptor we use to issue fsync. Instead, we generate the full path based on the FileTag in the sync request and use PathNameOpenFile to get the file descriptor. Author: Shawn Debnath, Thomas Munro Reviewed-by: Discussion: https://postgr.es/m/CAEepm=2gTANm=e3ARnJT=n0h8hf88wqmaZxk0JYkxw+b21fNrw@mail.gmail.com --- src/backend/access/transam/twophase.c | 1 + src/backend/access/transam/xact.c | 1 + src/backend/access/transam/xlog.c | 7 +- src/backend/commands/dbcommands.c | 7 +- src/backend/postmaster/checkpointer.c | 64 ++- src/backend/storage/Makefile | 2 +- src/backend/storage/buffer/bufmgr.c | 2 +- src/backend/storage/smgr/md.c | 846 ++++------------------------------ src/backend/storage/smgr/smgr.c | 55 +-- src/backend/storage/sync/Makefile | 17 + src/backend/storage/sync/sync.c | 638 +++++++++++++++++++++++++ src/backend/utils/init/postinit.c | 2 + src/include/postmaster/bgwriter.h | 8 +- src/include/storage/fd.h | 12 + src/include/storage/md.h | 51 ++ src/include/storage/segment.h | 28 ++ src/include/storage/smgr.h | 38 -- src/include/storage/sync.h | 86 ++++ 18 files changed, 988 insertions(+), 877 deletions(-) create mode 100644 src/backend/storage/sync/Makefile create mode 100644 src/backend/storage/sync/sync.c create mode 100644 src/include/storage/md.h create mode 100644 src/include/storage/segment.h create mode 100644 src/include/storage/sync.h diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c index 64679dd2de..80150467c7 100644 --- a/src/backend/access/transam/twophase.c +++ b/src/backend/access/transam/twophase.c @@ -98,6 +98,7 @@ #include "replication/walsender.h" #include "storage/fd.h" #include "storage/ipc.h" +#include "storage/md.h" #include "storage/predicate.h" #include "storage/proc.h" #include "storage/procarray.h" diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c index e93262975d..5384f62b34 100644 --- a/src/backend/access/transam/xact.c +++ b/src/backend/access/transam/xact.c @@ -50,6 +50,7 @@ #include "storage/fd.h" #include "storage/freespace.h" #include "storage/lmgr.h" +#include "storage/md.h" #include "storage/predicate.h" #include "storage/proc.h" #include "storage/procarray.h" diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c index ecd12fc53a..b2b154e77a 100644 --- a/src/backend/access/transam/xlog.c +++ b/src/backend/access/transam/xlog.c @@ -65,6 +65,7 @@ #include "storage/reinit.h" #include "storage/smgr.h" #include "storage/spin.h" +#include "storage/sync.h" #include "utils/builtins.h" #include "utils/guc.h" #include "utils/memutils.h" @@ -6986,7 +6987,7 @@ StartupXLOG(void) if (ArchiveRecoveryRequested && IsUnderPostmaster) { PublishStartupProcessInformation(); - SetForwardFsyncRequests(); + EnableSyncRequestForwarding(); SendPostmasterSignal(PMSIGNAL_RECOVERY_STARTED); bgwriterLaunched = true; } @@ -8616,7 +8617,7 @@ CreateCheckPoint(int flags) * the REDO pointer. Note that smgr must not do anything that'd have to * be undone if we decide no checkpoint is needed. */ - smgrpreckpt(); + SyncPreCheckpoint(); /* Begin filling in the checkpoint WAL record */ MemSet(&checkPoint, 0, sizeof(checkPoint)); @@ -8912,7 +8913,7 @@ CreateCheckPoint(int flags) /* * Let smgr do post-checkpoint cleanup (eg, deleting old files). */ - smgrpostckpt(); + SyncPostCheckpoint(); /* * Update the average distance between checkpoints if the prior checkpoint diff --git a/src/backend/commands/dbcommands.c b/src/backend/commands/dbcommands.c index d207cd899f..d553e2087c 100644 --- a/src/backend/commands/dbcommands.c +++ b/src/backend/commands/dbcommands.c @@ -53,6 +53,7 @@ #include "storage/fd.h" #include "storage/lmgr.h" #include "storage/ipc.h" +#include "storage/md.h" #include "storage/procarray.h" #include "storage/smgr.h" #include "utils/acl.h" @@ -940,11 +941,11 @@ dropdb(const char *dbname, bool missing_ok) * worse, it will delete files that belong to a newly created database * with the same OID. */ - ForgetDatabaseFsyncRequests(db_id); + ForgetDatabaseSyncRequests(db_id); /* * Force a checkpoint to make sure the checkpointer has received the - * message sent by ForgetDatabaseFsyncRequests. On Windows, this also + * message sent by ForgetDatabaseSyncRequests. On Windows, this also * ensures that background procs don't hold any open files, which would * cause rmdir() to fail. */ @@ -2149,7 +2150,7 @@ dbase_redo(XLogReaderState *record) DropDatabaseBuffers(xlrec->db_id); /* Also, clean out any fsync requests that might be pending in md.c */ - ForgetDatabaseFsyncRequests(xlrec->db_id); + ForgetDatabaseSyncRequests(xlrec->db_id); /* Clean out the xlog relcache too */ XLogDropDatabase(xlrec->db_id); diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c index fe96c41359..7529ea4bba 100644 --- a/src/backend/postmaster/checkpointer.c +++ b/src/backend/postmaster/checkpointer.c @@ -108,12 +108,38 @@ */ typedef struct { - RelFileNode rnode; - ForkNumber forknum; - BlockNumber segno; /* see md.c for special values */ - /* might add a real request-type field later; not needed yet */ + /* + * To reduce mmemory footprint, we combine the SyncRequestType and the + * SyncRequestHandler by splitting them into 4 bits each and storing them + * in an uint8. The type and handler values account for far fewer than + * 15 entries, so works just fine. + */ + uint8 sync_type_handler_combo; + + /* + * Currently, sync requests can be satisfied by information available in + * the FileIdentifier. In the future, this can be combined with a + * physical file descriptor or the full path to a file and put inside + * an union. + * + * This value is opaque to sync mechanism and is used to pass to callback + * handlers to retrieve path of the file to sync or to resolve forget + * requests. + */ + FileTag ftag; } CheckpointerRequest; +/* + * Handler occupies the higher 4 bits while type occupies the lower 4 in + * the uint8 combo storage. + */ +static uint8 sync_request_type_mask = 0x0F; +static uint8 sync_request_handler_mask = 0xF0; + +#define SYNC_TYPE_AND_HANDLER_COMBO(t, h) ((h) << 4 | (t)) +#define SYNC_REQUEST_TYPE_VALUE(v) (sync_request_type_mask & (v)) +#define SYNC_REQUEST_HANDLER_VALUE(v) ((sync_request_handler_mask & (v)) >> 4) + typedef struct { pid_t checkpointer_pid; /* PID (0 if not started) */ @@ -347,7 +373,7 @@ CheckpointerMain(void) /* * Process any requests or signals received recently. */ - AbsorbFsyncRequests(); + AbsorbSyncRequests(); if (got_SIGHUP) { @@ -676,7 +702,7 @@ CheckpointWriteDelay(int flags, double progress) UpdateSharedMemoryConfig(); } - AbsorbFsyncRequests(); + AbsorbSyncRequests(); absorb_counter = WRITES_PER_ABSORB; CheckArchiveTimeout(); @@ -701,7 +727,7 @@ CheckpointWriteDelay(int flags, double progress) * operations even when we don't sleep, to prevent overflow of the * fsync request queue. */ - AbsorbFsyncRequests(); + AbsorbSyncRequests(); absorb_counter = WRITES_PER_ABSORB; } } @@ -1063,7 +1089,7 @@ RequestCheckpoint(int flags) } /* - * ForwardFsyncRequest + * ForwardSyncRequest * Forward a file-fsync request from a backend to the checkpointer * * Whenever a backend is compelled to write directly to a relation @@ -1092,10 +1118,11 @@ RequestCheckpoint(int flags) * let the backend know by returning false. */ bool -ForwardFsyncRequest(RelFileNode rnode, ForkNumber forknum, BlockNumber segno) +ForwardSyncRequest(const FileTag *ftag, SyncRequestType type, + SyncRequestHandler handler) { CheckpointerRequest *request; - bool too_full; + bool too_full; if (!IsUnderPostmaster) return false; /* probably shouldn't even get here */ @@ -1130,9 +1157,8 @@ ForwardFsyncRequest(RelFileNode rnode, ForkNumber forknum, BlockNumber segno) /* OK, insert request */ request = &CheckpointerShmem->requests[CheckpointerShmem->num_requests++]; - request->rnode = rnode; - request->forknum = forknum; - request->segno = segno; + request->sync_type_handler_combo = SYNC_TYPE_AND_HANDLER_COMBO(type, handler); + request->ftag = *ftag; /* If queue is more than half full, nudge the checkpointer to empty it */ too_full = (CheckpointerShmem->num_requests >= @@ -1169,7 +1195,7 @@ CompactCheckpointerRequestQueue(void) struct CheckpointerSlotMapping { CheckpointerRequest request; - int slot; + int slot; }; int n, @@ -1263,8 +1289,8 @@ CompactCheckpointerRequestQueue(void) } /* - * AbsorbFsyncRequests - * Retrieve queued fsync requests and pass them to local smgr. + * AbsorbSyncRequests + * Retrieve queued sync requests and pass them to sync mechanism. * * This is exported because it must be called during CreateCheckPoint; * we have to be sure we have accepted all pending requests just before @@ -1272,7 +1298,7 @@ CompactCheckpointerRequestQueue(void) * non-checkpointer processes, do nothing if not checkpointer. */ void -AbsorbFsyncRequests(void) +AbsorbSyncRequests(void) { CheckpointerRequest *requests = NULL; CheckpointerRequest *request; @@ -1314,7 +1340,9 @@ AbsorbFsyncRequests(void) LWLockRelease(CheckpointerCommLock); for (request = requests; n > 0; request++, n--) - RememberFsyncRequest(request->rnode, request->forknum, request->segno); + RememberSyncRequest(&(request->ftag), + SYNC_REQUEST_TYPE_VALUE(request->sync_type_handler_combo), + SYNC_REQUEST_HANDLER_VALUE(request->sync_type_handler_combo)); END_CRIT_SECTION(); diff --git a/src/backend/storage/Makefile b/src/backend/storage/Makefile index bd2d272c6e..8376cdfca2 100644 --- a/src/backend/storage/Makefile +++ b/src/backend/storage/Makefile @@ -8,6 +8,6 @@ subdir = src/backend/storage top_builddir = ../../.. include $(top_builddir)/src/Makefile.global -SUBDIRS = buffer file freespace ipc large_object lmgr page smgr +SUBDIRS = buffer file freespace ipc large_object lmgr page smgr sync include $(top_srcdir)/src/backend/common.mk diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c index 273e2f385f..887023fc8a 100644 --- a/src/backend/storage/buffer/bufmgr.c +++ b/src/backend/storage/buffer/bufmgr.c @@ -2584,7 +2584,7 @@ CheckPointBuffers(int flags) BufferSync(flags); CheckpointStats.ckpt_sync_t = GetCurrentTimestamp(); TRACE_POSTGRESQL_BUFFER_CHECKPOINT_SYNC_START(); - smgrsync(); + ProcessSyncRequests(); CheckpointStats.ckpt_sync_end_t = GetCurrentTimestamp(); TRACE_POSTGRESQL_BUFFER_CHECKPOINT_DONE(); } diff --git a/src/backend/storage/smgr/md.c b/src/backend/storage/smgr/md.c index 2aba2dfe91..8cc9fb1614 100644 --- a/src/backend/storage/smgr/md.c +++ b/src/backend/storage/smgr/md.c @@ -29,45 +29,18 @@ #include "access/xlogutils.h" #include "access/xlog.h" #include "pgstat.h" -#include "portability/instr_time.h" #include "postmaster/bgwriter.h" #include "storage/fd.h" #include "storage/bufmgr.h" +#include "storage/md.h" #include "storage/relfilenode.h" +#include "storage/segment.h" #include "storage/smgr.h" +#include "storage/sync.h" #include "utils/hsearch.h" #include "utils/memutils.h" #include "pg_trace.h" - -/* intervals for calling AbsorbFsyncRequests in mdsync and mdpostckpt */ -#define FSYNCS_PER_ABSORB 10 -#define UNLINKS_PER_ABSORB 10 - -/* - * Special values for the segno arg to RememberFsyncRequest. - * - * Note that CompactCheckpointerRequestQueue assumes that it's OK to remove an - * fsync request from the queue if an identical, subsequent request is found. - * See comments there before making changes here. - */ -#define FORGET_RELATION_FSYNC (InvalidBlockNumber) -#define FORGET_DATABASE_FSYNC (InvalidBlockNumber-1) -#define UNLINK_RELATION_REQUEST (InvalidBlockNumber-2) - -/* - * On Windows, we have to interpret EACCES as possibly meaning the same as - * ENOENT, because if a file is unlinked-but-not-yet-gone on that platform, - * that's what you get. Ugh. This code is designed so that we don't - * actually believe these cases are okay without further evidence (namely, - * a pending fsync request getting canceled ... see mdsync). - */ -#ifndef WIN32 -#define FILE_POSSIBLY_DELETED(err) ((err) == ENOENT) -#else -#define FILE_POSSIBLY_DELETED(err) ((err) == ENOENT || (err) == EACCES) -#endif - /* * The magnetic disk storage manager keeps track of open file * descriptors in its own descriptor pool. This is done to make it @@ -114,53 +87,30 @@ typedef struct _MdfdVec static MemoryContext MdCxt; /* context for all MdfdVec objects */ +/* local routines */ +static void mdunlinkfork(RelFileNodeBackend rnode, ForkNumber forkNum, + bool isRedo); +static MdfdVec *mdopen(SMgrRelation reln, ForkNumber forknum, int behavior); +static void register_dirty_segment(SMgrRelation reln, ForkNumber forknum, + MdfdVec *seg); +static void register_unlink_segment(RelFileNodeBackend rnode, ForkNumber forknum, + SegmentNumber segno); +static void register_forget_request(RelFileNodeBackend rnode, ForkNumber forknum, + SegmentNumber segno); +static void _fdvec_resize(SMgrRelation reln, + ForkNumber forknum, + int nseg); +static MdfdVec *_mdfd_openseg(SMgrRelation reln, ForkNumber forkno, + BlockNumber segno, int oflags); +static MdfdVec *_mdfd_getseg(SMgrRelation reln, ForkNumber forkno, + BlockNumber blkno, bool skipFsync, int behavior); +static BlockNumber _mdnblocks(SMgrRelation reln, ForkNumber forknum, + MdfdVec *seg); + /* - * In some contexts (currently, standalone backends and the checkpointer) - * we keep track of pending fsync operations: we need to remember all relation - * segments that have been written since the last checkpoint, so that we can - * fsync them down to disk before completing the next checkpoint. This hash - * table remembers the pending operations. We use a hash table mostly as - * a convenient way of merging duplicate requests. - * - * We use a similar mechanism to remember no-longer-needed files that can - * be deleted after the next checkpoint, but we use a linked list instead of - * a hash table, because we don't expect there to be any duplicate requests. - * - * These mechanisms are only used for non-temp relations; we never fsync - * temp rels, nor do we need to postpone their deletion (see comments in - * mdunlink). - * - * (Regular backends do not track pending operations locally, but forward - * them to the checkpointer.) + * Segment handling behaviors */ -typedef uint16 CycleCtr; /* can be any convenient integer size */ - -typedef struct -{ - RelFileNode rnode; /* hash table key (must be first!) */ - CycleCtr cycle_ctr; /* mdsync_cycle_ctr of oldest request */ - /* requests[f] has bit n set if we need to fsync segment n of fork f */ - Bitmapset *requests[MAX_FORKNUM + 1]; - /* canceled[f] is true if we canceled fsyncs for fork "recently" */ - bool canceled[MAX_FORKNUM + 1]; -} PendingOperationEntry; - -typedef struct -{ - RelFileNode rnode; /* the dead relation to delete */ - CycleCtr cycle_ctr; /* mdckpt_cycle_ctr when request was made */ -} PendingUnlinkEntry; - -static HTAB *pendingOpsTable = NULL; -static List *pendingUnlinks = NIL; -static MemoryContext pendingOpsCxt; /* context for the above */ - -static CycleCtr mdsync_cycle_ctr = 0; -static CycleCtr mdckpt_cycle_ctr = 0; - - -/*** behavior for mdopen & _mdfd_getseg ***/ /* ereport if segment not present */ #define EXTENSION_FAIL (1 << 0) /* return NULL if segment not present */ @@ -179,26 +129,6 @@ static CycleCtr mdckpt_cycle_ctr = 0; #define EXTENSION_DONT_CHECK_SIZE (1 << 4) -/* local routines */ -static void mdunlinkfork(RelFileNodeBackend rnode, ForkNumber forkNum, - bool isRedo); -static MdfdVec *mdopen(SMgrRelation reln, ForkNumber forknum, int behavior); -static void register_dirty_segment(SMgrRelation reln, ForkNumber forknum, - MdfdVec *seg); -static void register_unlink(RelFileNodeBackend rnode); -static void _fdvec_resize(SMgrRelation reln, - ForkNumber forknum, - int nseg); -static char *_mdfd_segpath(SMgrRelation reln, ForkNumber forknum, - BlockNumber segno); -static MdfdVec *_mdfd_openseg(SMgrRelation reln, ForkNumber forkno, - BlockNumber segno, int oflags); -static MdfdVec *_mdfd_getseg(SMgrRelation reln, ForkNumber forkno, - BlockNumber blkno, bool skipFsync, int behavior); -static BlockNumber _mdnblocks(SMgrRelation reln, ForkNumber forknum, - MdfdVec *seg); - - /* * mdinit() -- Initialize private state for magnetic disk storage manager. */ @@ -208,64 +138,6 @@ mdinit(void) MdCxt = AllocSetContextCreate(TopMemoryContext, "MdSmgr", ALLOCSET_DEFAULT_SIZES); - - /* - * Create pending-operations hashtable if we need it. Currently, we need - * it if we are standalone (not under a postmaster) or if we are a startup - * or checkpointer auxiliary process. - */ - if (!IsUnderPostmaster || AmStartupProcess() || AmCheckpointerProcess()) - { - HASHCTL hash_ctl; - - /* - * XXX: The checkpointer needs to add entries to the pending ops table - * when absorbing fsync requests. That is done within a critical - * section, which isn't usually allowed, but we make an exception. It - * means that there's a theoretical possibility that you run out of - * memory while absorbing fsync requests, which leads to a PANIC. - * Fortunately the hash table is small so that's unlikely to happen in - * practice. - */ - pendingOpsCxt = AllocSetContextCreate(MdCxt, - "Pending ops context", - ALLOCSET_DEFAULT_SIZES); - MemoryContextAllowInCriticalSection(pendingOpsCxt, true); - - MemSet(&hash_ctl, 0, sizeof(hash_ctl)); - hash_ctl.keysize = sizeof(RelFileNode); - hash_ctl.entrysize = sizeof(PendingOperationEntry); - hash_ctl.hcxt = pendingOpsCxt; - pendingOpsTable = hash_create("Pending Ops Table", - 100L, - &hash_ctl, - HASH_ELEM | HASH_BLOBS | HASH_CONTEXT); - pendingUnlinks = NIL; - } -} - -/* - * In archive recovery, we rely on checkpointer to do fsyncs, but we will have - * already created the pendingOpsTable during initialization of the startup - * process. Calling this function drops the local pendingOpsTable so that - * subsequent requests will be forwarded to checkpointer. - */ -void -SetForwardFsyncRequests(void) -{ - /* Perform any pending fsyncs we may have queued up, then drop table */ - if (pendingOpsTable) - { - mdsync(); - hash_destroy(pendingOpsTable); - } - pendingOpsTable = NULL; - - /* - * We should not have any pending unlink requests, since mdunlink doesn't - * queue unlink requests when isRedo. - */ - Assert(pendingUnlinks == NIL); } /* @@ -380,16 +252,6 @@ mdcreate(SMgrRelation reln, ForkNumber forkNum, bool isRedo) void mdunlink(RelFileNodeBackend rnode, ForkNumber forkNum, bool isRedo) { - /* - * We have to clean out any pending fsync requests for the doomed - * relation, else the next mdsync() will fail. There can't be any such - * requests for a temp relation, though. We can send just one request - * even when deleting multiple forks, since the fsync queuing code accepts - * the "InvalidForkNumber = all forks" convention. - */ - if (!RelFileNodeBackendIsTemp(rnode)) - ForgetRelationFsyncRequests(rnode.node, forkNum); - /* Now do the per-fork work */ if (forkNum == InvalidForkNumber) { @@ -413,6 +275,11 @@ mdunlinkfork(RelFileNodeBackend rnode, ForkNumber forkNum, bool isRedo) */ if (isRedo || forkNum != MAIN_FORKNUM || RelFileNodeBackendIsTemp(rnode)) { + /* First, forget any pending sync requests for the first segment */ + if (!RelFileNodeBackendIsTemp(rnode)) + register_forget_request(rnode, forkNum, 0 /* first seg */); + + /* Next unlink the file */ ret = unlink(path); if (ret < 0 && errno != ENOENT) ereport(WARNING, @@ -442,7 +309,7 @@ mdunlinkfork(RelFileNodeBackend rnode, ForkNumber forkNum, bool isRedo) errmsg("could not truncate file \"%s\": %m", path))); /* Register request to unlink first segment later */ - register_unlink(rnode); + register_unlink_segment(rnode, forkNum, 0 /* first seg */); } /* @@ -459,6 +326,10 @@ mdunlinkfork(RelFileNodeBackend rnode, ForkNumber forkNum, bool isRedo) */ for (segno = 1;; segno++) { + /* Forget any pending sync requests for the segment before we unlink */ + if (!RelFileNodeBackendIsTemp(rnode)) + register_forget_request(rnode, forkNum, segno); + sprintf(segpath, "%s.%u", path, segno); if (unlink(segpath) < 0) { @@ -1004,385 +875,51 @@ mdimmedsync(SMgrRelation reln, ForkNumber forknum) } /* - * mdsync() -- Sync previous writes to stable storage. + * mdfilepath() + * + * Return the filename for the specified segment of the relation. The + * returned string is palloc'd. */ -void -mdsync(void) +char * +mdfilepath(const FileTag *ftag) { - static bool mdsync_in_progress = false; - - HASH_SEQ_STATUS hstat; - PendingOperationEntry *entry; - int absorb_counter; - - /* Statistics on sync times */ - int processed = 0; - instr_time sync_start, - sync_end, - sync_diff; - uint64 elapsed; - uint64 longest = 0; - uint64 total_elapsed = 0; - - /* - * This is only called during checkpoints, and checkpoints should only - * occur in processes that have created a pendingOpsTable. - */ - if (!pendingOpsTable) - elog(ERROR, "cannot sync without a pendingOpsTable"); + char *path, + *fullpath; /* - * If we are in the checkpointer, the sync had better include all fsync - * requests that were queued by backends up to this point. The tightest - * race condition that could occur is that a buffer that must be written - * and fsync'd for the checkpoint could have been dumped by a backend just - * before it was visited by BufferSync(). We know the backend will have - * queued an fsync request before clearing the buffer's dirtybit, so we - * are safe as long as we do an Absorb after completing BufferSync(). + * We can safely pass InvalidBackendId as we never expect to sync + * any segments for temporary relations. */ - AbsorbFsyncRequests(); + path = GetRelationPath(ftag->rnode.dbNode, ftag->rnode.spcNode, + ftag->rnode.relNode, InvalidBackendId, ftag->forknum); - /* - * To avoid excess fsync'ing (in the worst case, maybe a never-terminating - * checkpoint), we want to ignore fsync requests that are entered into the - * hashtable after this point --- they should be processed next time, - * instead. We use mdsync_cycle_ctr to tell old entries apart from new - * ones: new ones will have cycle_ctr equal to the incremented value of - * mdsync_cycle_ctr. - * - * In normal circumstances, all entries present in the table at this point - * will have cycle_ctr exactly equal to the current (about to be old) - * value of mdsync_cycle_ctr. However, if we fail partway through the - * fsync'ing loop, then older values of cycle_ctr might remain when we - * come back here to try again. Repeated checkpoint failures would - * eventually wrap the counter around to the point where an old entry - * might appear new, causing us to skip it, possibly allowing a checkpoint - * to succeed that should not have. To forestall wraparound, any time the - * previous mdsync() failed to complete, run through the table and - * forcibly set cycle_ctr = mdsync_cycle_ctr. - * - * Think not to merge this loop with the main loop, as the problem is - * exactly that that loop may fail before having visited all the entries. - * From a performance point of view it doesn't matter anyway, as this path - * will never be taken in a system that's functioning normally. - */ - if (mdsync_in_progress) + if (ftag->segno > 0 && ftag->segno != InvalidSegmentNumber) { - /* prior try failed, so update any stale cycle_ctr values */ - hash_seq_init(&hstat, pendingOpsTable); - while ((entry = (PendingOperationEntry *) hash_seq_search(&hstat)) != NULL) - { - entry->cycle_ctr = mdsync_cycle_ctr; - } + fullpath = psprintf("%s.%u", path, ftag->segno); + pfree(path); } + else + fullpath = path; - /* Advance counter so that new hashtable entries are distinguishable */ - mdsync_cycle_ctr++; - - /* Set flag to detect failure if we don't reach the end of the loop */ - mdsync_in_progress = true; - - /* Now scan the hashtable for fsync requests to process */ - absorb_counter = FSYNCS_PER_ABSORB; - hash_seq_init(&hstat, pendingOpsTable); - while ((entry = (PendingOperationEntry *) hash_seq_search(&hstat)) != NULL) - { - ForkNumber forknum; - - /* - * If the entry is new then don't process it this time; it might - * contain multiple fsync-request bits, but they are all new. Note - * "continue" bypasses the hash-remove call at the bottom of the loop. - */ - if (entry->cycle_ctr == mdsync_cycle_ctr) - continue; - - /* Else assert we haven't missed it */ - Assert((CycleCtr) (entry->cycle_ctr + 1) == mdsync_cycle_ctr); - - /* - * Scan over the forks and segments represented by the entry. - * - * The bitmap manipulations are slightly tricky, because we can call - * AbsorbFsyncRequests() inside the loop and that could result in - * bms_add_member() modifying and even re-palloc'ing the bitmapsets. - * So we detach it, but if we fail we'll merge it with any new - * requests that have arrived in the meantime. - */ - for (forknum = 0; forknum <= MAX_FORKNUM; forknum++) - { - Bitmapset *requests = entry->requests[forknum]; - int segno; - - entry->requests[forknum] = NULL; - entry->canceled[forknum] = false; - - segno = -1; - while ((segno = bms_next_member(requests, segno)) >= 0) - { - int failures; - - /* - * If fsync is off then we don't have to bother opening the - * file at all. (We delay checking until this point so that - * changing fsync on the fly behaves sensibly.) - */ - if (!enableFsync) - continue; - - /* - * If in checkpointer, we want to absorb pending requests - * every so often to prevent overflow of the fsync request - * queue. It is unspecified whether newly-added entries will - * be visited by hash_seq_search, but we don't care since we - * don't need to process them anyway. - */ - if (--absorb_counter <= 0) - { - AbsorbFsyncRequests(); - absorb_counter = FSYNCS_PER_ABSORB; - } - - /* - * The fsync table could contain requests to fsync segments - * that have been deleted (unlinked) by the time we get to - * them. Rather than just hoping an ENOENT (or EACCES on - * Windows) error can be ignored, what we do on error is - * absorb pending requests and then retry. Since mdunlink() - * queues a "cancel" message before actually unlinking, the - * fsync request is guaranteed to be marked canceled after the - * absorb if it really was this case. DROP DATABASE likewise - * has to tell us to forget fsync requests before it starts - * deletions. - */ - for (failures = 0;; failures++) /* loop exits at "break" */ - { - SMgrRelation reln; - MdfdVec *seg; - char *path; - int save_errno; - - /* - * Find or create an smgr hash entry for this relation. - * This may seem a bit unclean -- md calling smgr? But - * it's really the best solution. It ensures that the - * open file reference isn't permanently leaked if we get - * an error here. (You may say "but an unreferenced - * SMgrRelation is still a leak!" Not really, because the - * only case in which a checkpoint is done by a process - * that isn't about to shut down is in the checkpointer, - * and it will periodically do smgrcloseall(). This fact - * justifies our not closing the reln in the success path - * either, which is a good thing since in non-checkpointer - * cases we couldn't safely do that.) - */ - reln = smgropen(entry->rnode, InvalidBackendId); - - /* Attempt to open and fsync the target segment */ - seg = _mdfd_getseg(reln, forknum, - (BlockNumber) segno * (BlockNumber) RELSEG_SIZE, - false, - EXTENSION_RETURN_NULL - | EXTENSION_DONT_CHECK_SIZE); - - INSTR_TIME_SET_CURRENT(sync_start); - - if (seg != NULL && - FileSync(seg->mdfd_vfd, WAIT_EVENT_DATA_FILE_SYNC) >= 0) - { - /* Success; update statistics about sync timing */ - INSTR_TIME_SET_CURRENT(sync_end); - sync_diff = sync_end; - INSTR_TIME_SUBTRACT(sync_diff, sync_start); - elapsed = INSTR_TIME_GET_MICROSEC(sync_diff); - if (elapsed > longest) - longest = elapsed; - total_elapsed += elapsed; - processed++; - requests = bms_del_member(requests, segno); - if (log_checkpoints) - elog(DEBUG1, "checkpoint sync: number=%d file=%s time=%.3f msec", - processed, - FilePathName(seg->mdfd_vfd), - (double) elapsed / 1000); - - break; /* out of retry loop */ - } - - /* Compute file name for use in message */ - save_errno = errno; - path = _mdfd_segpath(reln, forknum, (BlockNumber) segno); - errno = save_errno; - - /* - * It is possible that the relation has been dropped or - * truncated since the fsync request was entered. - * Therefore, allow ENOENT, but only if we didn't fail - * already on this file. This applies both for - * _mdfd_getseg() and for FileSync, since fd.c might have - * closed the file behind our back. - * - * XXX is there any point in allowing more than one retry? - * Don't see one at the moment, but easy to change the - * test here if so. - */ - if (!FILE_POSSIBLY_DELETED(errno) || - failures > 0) - { - Bitmapset *new_requests; - - /* - * We need to merge these unsatisfied requests with - * any others that have arrived since we started. - */ - new_requests = entry->requests[forknum]; - entry->requests[forknum] = - bms_join(new_requests, requests); - - errno = save_errno; - ereport(data_sync_elevel(ERROR), - (errcode_for_file_access(), - errmsg("could not fsync file \"%s\": %m", - path))); - } - else - ereport(DEBUG1, - (errcode_for_file_access(), - errmsg("could not fsync file \"%s\" but retrying: %m", - path))); - pfree(path); - - /* - * Absorb incoming requests and check to see if a cancel - * arrived for this relation fork. - */ - AbsorbFsyncRequests(); - absorb_counter = FSYNCS_PER_ABSORB; /* might as well... */ - - if (entry->canceled[forknum]) - break; - } /* end retry loop */ - } - bms_free(requests); - } - - /* - * We've finished everything that was requested before we started to - * scan the entry. If no new requests have been inserted meanwhile, - * remove the entry. Otherwise, update its cycle counter, as all the - * requests now in it must have arrived during this cycle. - */ - for (forknum = 0; forknum <= MAX_FORKNUM; forknum++) - { - if (entry->requests[forknum] != NULL) - break; - } - if (forknum <= MAX_FORKNUM) - entry->cycle_ctr = mdsync_cycle_ctr; - else - { - /* Okay to remove it */ - if (hash_search(pendingOpsTable, &entry->rnode, - HASH_REMOVE, NULL) == NULL) - elog(ERROR, "pendingOpsTable corrupted"); - } - } /* end loop over hashtable entries */ - - /* Return sync performance metrics for report at checkpoint end */ - CheckpointStats.ckpt_sync_rels = processed; - CheckpointStats.ckpt_longest_sync = longest; - CheckpointStats.ckpt_agg_sync_time = total_elapsed; - - /* Flag successful completion of mdsync */ - mdsync_in_progress = false; -} - -/* - * mdpreckpt() -- Do pre-checkpoint work - * - * To distinguish unlink requests that arrived before this checkpoint - * started from those that arrived during the checkpoint, we use a cycle - * counter similar to the one we use for fsync requests. That cycle - * counter is incremented here. - * - * This must be called *before* the checkpoint REDO point is determined. - * That ensures that we won't delete files too soon. - * - * Note that we can't do anything here that depends on the assumption - * that the checkpoint will be completed. - */ -void -mdpreckpt(void) -{ - /* - * Any unlink requests arriving after this point will be assigned the next - * cycle counter, and won't be unlinked until next checkpoint. - */ - mdckpt_cycle_ctr++; + return fullpath; } /* - * mdpostckpt() -- Do post-checkpoint work + * mdfiletagmatches() * - * Remove any lingering files that can now be safely removed. + * Returns true if the predicate tag matches with the file tag. */ -void -mdpostckpt(void) +bool +mdfiletagmatches(const FileTag *ftag, const FileTag *predicate, + SyncRequestType type) { - int absorb_counter; + /* Today, we only do matching for hierarchy (forget database) requests */ + Assert(type == SYNC_FORGET_HIERARCHY_REQUEST); - absorb_counter = UNLINKS_PER_ABSORB; - while (pendingUnlinks != NIL) - { - PendingUnlinkEntry *entry = (PendingUnlinkEntry *) linitial(pendingUnlinks); - char *path; + if (type == SYNC_FORGET_HIERARCHY_REQUEST) + return ftag->rnode.dbNode == predicate->rnode.dbNode; - /* - * New entries are appended to the end, so if the entry is new we've - * reached the end of old entries. - * - * Note: if just the right number of consecutive checkpoints fail, we - * could be fooled here by cycle_ctr wraparound. However, the only - * consequence is that we'd delay unlinking for one more checkpoint, - * which is perfectly tolerable. - */ - if (entry->cycle_ctr == mdckpt_cycle_ctr) - break; - - /* Unlink the file */ - path = relpathperm(entry->rnode, MAIN_FORKNUM); - if (unlink(path) < 0) - { - /* - * There's a race condition, when the database is dropped at the - * same time that we process the pending unlink requests. If the - * DROP DATABASE deletes the file before we do, we will get ENOENT - * here. rmtree() also has to ignore ENOENT errors, to deal with - * the possibility that we delete the file first. - */ - if (errno != ENOENT) - ereport(WARNING, - (errcode_for_file_access(), - errmsg("could not remove file \"%s\": %m", path))); - } - pfree(path); - - /* And remove the list entry */ - pendingUnlinks = list_delete_first(pendingUnlinks); - pfree(entry); - - /* - * As in mdsync, we don't want to stop absorbing fsync requests for a - * long time when there are many deletions to be done. We can safely - * call AbsorbFsyncRequests() at this point in the loop (note it might - * try to delete list entries). - */ - if (--absorb_counter <= 0) - { - AbsorbFsyncRequests(); - absorb_counter = UNLINKS_PER_ABSORB; - } - } + return false; } /* @@ -1397,19 +934,16 @@ mdpostckpt(void) static void register_dirty_segment(SMgrRelation reln, ForkNumber forknum, MdfdVec *seg) { + FileTag tag; + + INIT_FILETAG(tag, reln->smgr_rnode.node, forknum, seg->mdfd_segno); + /* Temp relations should never be fsync'd */ Assert(!SmgrIsTemp(reln)); - if (pendingOpsTable) - { - /* push it into local pending-ops table */ - RememberFsyncRequest(reln->smgr_rnode.node, forknum, seg->mdfd_segno); - } - else + if (!RegisterSyncRequest(&tag, SYNC_REQUEST, SYNC_HANDLER_MD, + false /*retryOnError*/)) { - if (ForwardFsyncRequest(reln->smgr_rnode.node, forknum, seg->mdfd_segno)) - return; /* passed it off successfully */ - ereport(DEBUG1, (errmsg("could not forward fsync request because request queue is full"))); @@ -1423,254 +957,54 @@ register_dirty_segment(SMgrRelation reln, ForkNumber forknum, MdfdVec *seg) /* * register_unlink() -- Schedule a file to be deleted after next checkpoint - * - * We don't bother passing in the fork number, because this is only used - * with main forks. - * - * As with register_dirty_segment, this could involve either a local or - * a remote pending-ops table. */ static void -register_unlink(RelFileNodeBackend rnode) +register_unlink_segment(RelFileNodeBackend rnode, ForkNumber forknum, + SegmentNumber segno) { + FileTag tag; + + INIT_FILETAG(tag, rnode.node, forknum, segno); + /* Should never be used with temp relations */ Assert(!RelFileNodeBackendIsTemp(rnode)); - if (pendingOpsTable) - { - /* push it into local pending-ops table */ - RememberFsyncRequest(rnode.node, MAIN_FORKNUM, - UNLINK_RELATION_REQUEST); - } - else - { - /* - * Notify the checkpointer about it. If we fail to queue the request - * message, we have to sleep and try again, because we can't simply - * delete the file now. Ugly, but hopefully won't happen often. - * - * XXX should we just leave the file orphaned instead? - */ - Assert(IsUnderPostmaster); - while (!ForwardFsyncRequest(rnode.node, MAIN_FORKNUM, - UNLINK_RELATION_REQUEST)) - pg_usleep(10000L); /* 10 msec seems a good number */ - } + RegisterSyncRequest(&tag, SYNC_UNLINK_REQUEST, SYNC_HANDLER_MD, + true /*retryOnError*/); } /* - * RememberFsyncRequest() -- callback from checkpointer side of fsync request - * - * We stuff fsync requests into the local hash table for execution - * during the checkpointer's next checkpoint. UNLINK requests go into a - * separate linked list, however, because they get processed separately. - * - * The range of possible segment numbers is way less than the range of - * BlockNumber, so we can reserve high values of segno for special purposes. - * We define three: - * - FORGET_RELATION_FSYNC means to cancel pending fsyncs for a relation, - * either for one fork, or all forks if forknum is InvalidForkNumber - * - FORGET_DATABASE_FSYNC means to cancel pending fsyncs for a whole database - * - UNLINK_RELATION_REQUEST is a request to delete the file after the next - * checkpoint. - * Note also that we're assuming real segment numbers don't exceed INT_MAX. - * - * (Handling FORGET_DATABASE_FSYNC requests is a tad slow because the hash - * table has to be searched linearly, but dropping a database is a pretty - * heavyweight operation anyhow, so we'll live with it.) + * register_forget_request() -- forget any fsyncs for a relation fork's segment */ -void -RememberFsyncRequest(RelFileNode rnode, ForkNumber forknum, BlockNumber segno) +static void +register_forget_request(RelFileNodeBackend rnode, ForkNumber forknum, + SegmentNumber segno) { - Assert(pendingOpsTable); + FileTag tag; - if (segno == FORGET_RELATION_FSYNC) - { - /* Remove any pending requests for the relation (one or all forks) */ - PendingOperationEntry *entry; - - entry = (PendingOperationEntry *) hash_search(pendingOpsTable, - &rnode, - HASH_FIND, - NULL); - if (entry) - { - /* - * We can't just delete the entry since mdsync could have an - * active hashtable scan. Instead we delete the bitmapsets; this - * is safe because of the way mdsync is coded. We also set the - * "canceled" flags so that mdsync can tell that a cancel arrived - * for the fork(s). - */ - if (forknum == InvalidForkNumber) - { - /* remove requests for all forks */ - for (forknum = 0; forknum <= MAX_FORKNUM; forknum++) - { - bms_free(entry->requests[forknum]); - entry->requests[forknum] = NULL; - entry->canceled[forknum] = true; - } - } - else - { - /* remove requests for single fork */ - bms_free(entry->requests[forknum]); - entry->requests[forknum] = NULL; - entry->canceled[forknum] = true; - } - } - } - else if (segno == FORGET_DATABASE_FSYNC) - { - /* Remove any pending requests for the entire database */ - HASH_SEQ_STATUS hstat; - PendingOperationEntry *entry; - ListCell *cell, - *prev, - *next; - - /* Remove fsync requests */ - hash_seq_init(&hstat, pendingOpsTable); - while ((entry = (PendingOperationEntry *) hash_seq_search(&hstat)) != NULL) - { - if (entry->rnode.dbNode == rnode.dbNode) - { - /* remove requests for all forks */ - for (forknum = 0; forknum <= MAX_FORKNUM; forknum++) - { - bms_free(entry->requests[forknum]); - entry->requests[forknum] = NULL; - entry->canceled[forknum] = true; - } - } - } - - /* Remove unlink requests */ - prev = NULL; - for (cell = list_head(pendingUnlinks); cell; cell = next) - { - PendingUnlinkEntry *entry = (PendingUnlinkEntry *) lfirst(cell); + INIT_FILETAG(tag, rnode.node, forknum, segno); - next = lnext(cell); - if (entry->rnode.dbNode == rnode.dbNode) - { - pendingUnlinks = list_delete_cell(pendingUnlinks, cell, prev); - pfree(entry); - } - else - prev = cell; - } - } - else if (segno == UNLINK_RELATION_REQUEST) - { - /* Unlink request: put it in the linked list */ - MemoryContext oldcxt = MemoryContextSwitchTo(pendingOpsCxt); - PendingUnlinkEntry *entry; - - /* PendingUnlinkEntry doesn't store forknum, since it's always MAIN */ - Assert(forknum == MAIN_FORKNUM); - - entry = palloc(sizeof(PendingUnlinkEntry)); - entry->rnode = rnode; - entry->cycle_ctr = mdckpt_cycle_ctr; - - pendingUnlinks = lappend(pendingUnlinks, entry); - - MemoryContextSwitchTo(oldcxt); - } - else - { - /* Normal case: enter a request to fsync this segment */ - MemoryContext oldcxt = MemoryContextSwitchTo(pendingOpsCxt); - PendingOperationEntry *entry; - bool found; - - entry = (PendingOperationEntry *) hash_search(pendingOpsTable, - &rnode, - HASH_ENTER, - &found); - /* if new entry, initialize it */ - if (!found) - { - entry->cycle_ctr = mdsync_cycle_ctr; - MemSet(entry->requests, 0, sizeof(entry->requests)); - MemSet(entry->canceled, 0, sizeof(entry->canceled)); - } - - /* - * NB: it's intentional that we don't change cycle_ctr if the entry - * already exists. The cycle_ctr must represent the oldest fsync - * request that could be in the entry. - */ - - entry->requests[forknum] = bms_add_member(entry->requests[forknum], - (int) segno); - - MemoryContextSwitchTo(oldcxt); - } -} - -/* - * ForgetRelationFsyncRequests -- forget any fsyncs for a relation fork - * - * forknum == InvalidForkNumber means all forks, although this code doesn't - * actually know that, since it's just forwarding the request elsewhere. - */ -void -ForgetRelationFsyncRequests(RelFileNode rnode, ForkNumber forknum) -{ - if (pendingOpsTable) - { - /* standalone backend or startup process: fsync state is local */ - RememberFsyncRequest(rnode, forknum, FORGET_RELATION_FSYNC); - } - else if (IsUnderPostmaster) - { - /* - * Notify the checkpointer about it. If we fail to queue the cancel - * message, we have to sleep and try again ... ugly, but hopefully - * won't happen often. - * - * XXX should we CHECK_FOR_INTERRUPTS in this loop? Escaping with an - * error would leave the no-longer-used file still present on disk, - * which would be bad, so I'm inclined to assume that the checkpointer - * will always empty the queue soon. - */ - while (!ForwardFsyncRequest(rnode, forknum, FORGET_RELATION_FSYNC)) - pg_usleep(10000L); /* 10 msec seems a good number */ - - /* - * Note we don't wait for the checkpointer to actually absorb the - * cancel message; see mdsync() for the implications. - */ - } + RegisterSyncRequest(&tag, SYNC_FORGET_REQUEST, SYNC_HANDLER_MD, + true /*retryOnError*/); } /* * ForgetDatabaseFsyncRequests -- forget any fsyncs and unlinks for a DB */ void -ForgetDatabaseFsyncRequests(Oid dbid) +ForgetDatabaseSyncRequests(Oid dbid) { + FileTag tag; RelFileNode rnode; rnode.dbNode = dbid; rnode.spcNode = 0; rnode.relNode = 0; - if (pendingOpsTable) - { - /* standalone backend or startup process: fsync state is local */ - RememberFsyncRequest(rnode, InvalidForkNumber, FORGET_DATABASE_FSYNC); - } - else if (IsUnderPostmaster) - { - /* see notes in ForgetRelationFsyncRequests */ - while (!ForwardFsyncRequest(rnode, InvalidForkNumber, - FORGET_DATABASE_FSYNC)) - pg_usleep(10000L); /* 10 msec seems a good number */ - } + INIT_FILETAG(tag, rnode, InvalidForkNumber, InvalidSegmentNumber); + + RegisterSyncRequest(&tag, SYNC_FORGET_HIERARCHY_REQUEST, SYNC_HANDLER_MD, + true /*retryOnError*/); } /* diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c index 0c0bba4ab3..39f4fed25e 100644 --- a/src/backend/storage/smgr/smgr.c +++ b/src/backend/storage/smgr/smgr.c @@ -20,6 +20,7 @@ #include "commands/tablespace.h" #include "storage/bufmgr.h" #include "storage/ipc.h" +#include "storage/md.h" #include "storage/smgr.h" #include "utils/hsearch.h" #include "utils/inval.h" @@ -59,12 +60,8 @@ typedef struct f_smgr void (*smgr_truncate) (SMgrRelation reln, ForkNumber forknum, BlockNumber nblocks); void (*smgr_immedsync) (SMgrRelation reln, ForkNumber forknum); - void (*smgr_pre_ckpt) (void); /* may be NULL */ - void (*smgr_sync) (void); /* may be NULL */ - void (*smgr_post_ckpt) (void); /* may be NULL */ } f_smgr; - static const f_smgr smgrsw[] = { /* magnetic disk */ { @@ -82,15 +79,11 @@ static const f_smgr smgrsw[] = { .smgr_nblocks = mdnblocks, .smgr_truncate = mdtruncate, .smgr_immedsync = mdimmedsync, - .smgr_pre_ckpt = mdpreckpt, - .smgr_sync = mdsync, - .smgr_post_ckpt = mdpostckpt } }; static const int NSmgr = lengthof(smgrsw); - /* * Each backend has a hashtable that stores all extant SMgrRelation objects. * In addition, "unowned" SMgrRelation objects are chained together in a list. @@ -751,52 +744,6 @@ smgrimmedsync(SMgrRelation reln, ForkNumber forknum) smgrsw[reln->smgr_which].smgr_immedsync(reln, forknum); } - -/* - * smgrpreckpt() -- Prepare for checkpoint. - */ -void -smgrpreckpt(void) -{ - int i; - - for (i = 0; i < NSmgr; i++) - { - if (smgrsw[i].smgr_pre_ckpt) - smgrsw[i].smgr_pre_ckpt(); - } -} - -/* - * smgrsync() -- Sync files to disk during checkpoint. - */ -void -smgrsync(void) -{ - int i; - - for (i = 0; i < NSmgr; i++) - { - if (smgrsw[i].smgr_sync) - smgrsw[i].smgr_sync(); - } -} - -/* - * smgrpostckpt() -- Post-checkpoint cleanup. - */ -void -smgrpostckpt(void) -{ - int i; - - for (i = 0; i < NSmgr; i++) - { - if (smgrsw[i].smgr_post_ckpt) - smgrsw[i].smgr_post_ckpt(); - } -} - /* * AtEOXact_SMgr * diff --git a/src/backend/storage/sync/Makefile b/src/backend/storage/sync/Makefile new file mode 100644 index 0000000000..cfc60cadb4 --- /dev/null +++ b/src/backend/storage/sync/Makefile @@ -0,0 +1,17 @@ +#------------------------------------------------------------------------- +# +# Makefile-- +# Makefile for storage/sync +# +# IDENTIFICATION +# src/backend/storage/sync/Makefile +# +#------------------------------------------------------------------------- + +subdir = src/backend/storage/sync +top_builddir = ../../../.. +include $(top_builddir)/src/Makefile.global + +OBJS = sync.o + +include $(top_srcdir)/src/backend/common.mk diff --git a/src/backend/storage/sync/sync.c b/src/backend/storage/sync/sync.c new file mode 100644 index 0000000000..5f7db69e8a --- /dev/null +++ b/src/backend/storage/sync/sync.c @@ -0,0 +1,638 @@ +/*------------------------------------------------------------------------- + * + * sync.c + * File synchronization management code. + * + * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * + * IDENTIFICATION + * src/backend/storage/sync/sync.c + * + *------------------------------------------------------------------------- + */ +#include "postgres.h" + +#include +#include +#include + +#include "miscadmin.h" +#include "pgstat.h" +#include "access/xlogutils.h" +#include "access/xlog.h" +#include "commands/tablespace.h" +#include "portability/instr_time.h" +#include "postmaster/bgwriter.h" +#include "storage/bufmgr.h" +#include "storage/ipc.h" +#include "storage/md.h" +#include "utils/hsearch.h" +#include "utils/memutils.h" +#include "utils/inval.h" + +static MemoryContext pendingOpsCxt; /* context for the pending ops state */ + +/* + * In some contexts (currently, standalone backends and the checkpointer) + * we keep track of pending fsync operations: we need to remember all relation + * segments that have been written since the last checkpoint, so that we can + * fsync them down to disk before completing the next checkpoint. This hash + * table remembers the pending operations. We use a hash table mostly as + * a convenient way of merging duplicate requests. + * + * We use a similar mechanism to remember no-longer-needed files that can + * be deleted after the next checkpoint, but we use a linked list instead of + * a hash table, because we don't expect there to be any duplicate requests. + * + * These mechanisms are only used for non-temp relations; we never fsync + * temp rels, nor do we need to postpone their deletion (see comments in + * mdunlink). + * + * (Regular backends do not track pending operations locally, but forward + * them to the checkpointer.) + */ +typedef uint16 CycleCtr; /* can be any convenient integer size */ + +typedef struct +{ + FileTag ftag; /* hash table key (must be first!) */ + SyncRequestHandler handler; /* request resolution handler */ + CycleCtr cycle_ctr; /* sync_cycle_ctr of oldest request */ + bool canceled; /* canceled is true if we canceled "recently" */ +} PendingFsyncEntry; + +typedef struct +{ + FileTag ftag; /* tag for relation file to delete */ + SyncRequestHandler handler; /* request resolution handler */ + CycleCtr cycle_ctr; /* checkpoint_cycle_ctr when request was made */ +} PendingUnlinkEntry; + +static HTAB *pendingOps = NULL; +static List *pendingUnlinks = NIL; +static MemoryContext pendingOpsCxt; /* context for the above */ + +static CycleCtr sync_cycle_ctr = 0; +static CycleCtr checkpoint_cycle_ctr = 0; + +/* Intervals for calling AbsorbFsyncRequests */ +#define FSYNCS_PER_ABSORB 10 +#define UNLINKS_PER_ABSORB 10 + +/* + * This struct of function pointers defines the API between smgr.c and + * any individual storage manager module. Note that smgr subfunctions are + * generally expected to report problems via elog(ERROR). An exception is + * that smgr_unlink should use elog(WARNING), rather than erroring out, + * because we normally unlink relations during post-commit/abort cleanup, + * and so it's too late to raise an error. Also, various conditions that + * would normally be errors should be allowed during bootstrap and/or WAL + * recovery --- see comments in md.c for details. + */ +typedef struct f_sync +{ + char* (*sync_filepath) (const FileTag *ftag); + bool (*sync_filetagmatches) (const FileTag *ftag, + const FileTag *predicate, SyncRequestType type); +} f_sync; + +static const f_sync syncsw[] = { + /* magnetic disk */ + { + .sync_filepath = mdfilepath, + .sync_filetagmatches = mdfiletagmatches + } +}; + +/* + * Initialize data structures for the file sync tracking. + */ +void +InitSync(void) +{ + /* + * Create pending-operations hashtable if we need it. Currently, we need + * it if we are standalone (not under a postmaster) or if we are a startup + * or checkpointer auxiliary process. + */ + if (!IsUnderPostmaster || AmStartupProcess() || AmCheckpointerProcess()) + { + HASHCTL hash_ctl; + + /* + * XXX: The checkpointer needs to add entries to the pending ops table + * when absorbing fsync requests. That is done within a critical + * section, which isn't usually allowed, but we make an exception. It + * means that there's a theoretical possibility that you run out of + * memory while absorbing fsync requests, which leads to a PANIC. + * Fortunately the hash table is small so that's unlikely to happen in + * practice. + */ + pendingOpsCxt = AllocSetContextCreate(TopMemoryContext, + "Pending ops context", + ALLOCSET_DEFAULT_SIZES); + MemoryContextAllowInCriticalSection(pendingOpsCxt, true); + + MemSet(&hash_ctl, 0, sizeof(hash_ctl)); + hash_ctl.keysize = sizeof(FileTag); + hash_ctl.entrysize = sizeof(PendingFsyncEntry); + hash_ctl.hcxt = pendingOpsCxt; + pendingOps = hash_create("Pending Ops Table", + 100L, + &hash_ctl, + HASH_ELEM | HASH_BLOBS | HASH_CONTEXT); + pendingUnlinks = NIL; + } + +} + +/* + * SyncPreCheckpoint() -- Do pre-checkpoint work + * + * To distinguish unlink requests that arrived before this checkpoint + * started from those that arrived during the checkpoint, we use a cycle + * counter similar to the one we use for fsync requests. That cycle + * counter is incremented here. + * + * This must be called *before* the checkpoint REDO point is determined. + * That ensures that we won't delete files too soon. + * + * Note that we can't do anything here that depends on the assumption + * that the checkpoint will be completed. + */ +void +SyncPreCheckpoint(void) +{ + /* + * Any unlink requests arriving after this point will be assigned the next + * cycle counter, and won't be unlinked until next checkpoint. + */ + checkpoint_cycle_ctr++; +} + +/* + * SyncPostCheckpoint() -- Do post-checkpoint work + * + * Remove any lingering files that can now be safely removed. + */ +void +SyncPostCheckpoint(void) +{ + int absorb_counter; + + absorb_counter = UNLINKS_PER_ABSORB; + while (pendingUnlinks != NIL) + { + PendingUnlinkEntry *entry = (PendingUnlinkEntry *) linitial(pendingUnlinks); + char *path; + + /* + * New entries are appended to the end, so if the entry is new we've + * reached the end of old entries. + * + * Note: if just the right number of consecutive checkpoints fail, we + * could be fooled here by cycle_ctr wraparound. However, the only + * consequence is that we'd delay unlinking for one more checkpoint, + * which is perfectly tolerable. + */ + if (entry->cycle_ctr == checkpoint_cycle_ctr) + break; + + /* Unlink the file */ + path = syncsw[entry->handler].sync_filepath(&(entry->ftag)); + + if (unlink(path) < 0) + { + /* + * There's a race condition, when the database is dropped at the + * same time that we process the pending unlink requests. If the + * DROP DATABASE deletes the file before we do, we will get ENOENT + * here. rmtree() also has to ignore ENOENT errors, to deal with + * the possibility that we delete the file first. + */ + if (errno != ENOENT) + ereport(WARNING, + (errcode_for_file_access(), + errmsg("could not remove file \"%s\": %m", path))); + } + pfree(path); + + /* And remove the list entry */ + pendingUnlinks = list_delete_first(pendingUnlinks); + pfree(entry); + + /* + * As in ProcessFsyncRequests, we don't want to stop absorbing fsync + * requests for along time when there are many deletions to be done. We + * can safelycall AbsorbFsyncRequests() at this point in the loop + * (note it might try to delete list entries). + */ + if (--absorb_counter <= 0) + { + AbsorbSyncRequests(); + absorb_counter = UNLINKS_PER_ABSORB; + } + } +} + +/* + + * ProcessSyncRequests() -- Process queued fsync requests. + */ +void +ProcessSyncRequests(void) +{ + static bool sync_in_progress = false; + + HASH_SEQ_STATUS hstat; + PendingFsyncEntry *entry; + int absorb_counter; + + /* Statistics on sync times */ + int processed = 0; + instr_time sync_start, + sync_end, + sync_diff; + uint64 elapsed; + uint64 longest = 0; + uint64 total_elapsed = 0; + + /* + * This is only called during checkpoints, and checkpoints should only + * occur in processes that have created a pendingOps. + */ + if (!pendingOps) + elog(ERROR, "cannot sync without a pendingOps table"); + + /* + * If we are in the checkpointer, the sync had better include all fsync + * requests that were queued by backends up to this point. The tightest + * race condition that could occur is that a buffer that must be written + * and fsync'd for the checkpoint could have been dumped by a backend just + * before it was visited by BufferSync(). We know the backend will have + * queued an fsync request before clearing the buffer's dirtybit, so we + * are safe as long as we do an Absorb after completing BufferSync(). + */ + AbsorbSyncRequests(); + + /* + * To avoid excess fsync'ing (in the worst case, maybe a never-terminating + * checkpoint), we want to ignore fsync requests that are entered into the + * hashtable after this point --- they should be processed next time, + * instead. We use sync_cycle_ctr to tell old entries apart from new + * ones: new ones will have cycle_ctr equal to the incremented value of + * sync_cycle_ctr. + * + * In normal circumstances, all entries present in the table at this point + * will have cycle_ctr exactly equal to the current (about to be old) + * value of sync_cycle_ctr. However, if we fail partway through the + * fsync'ing loop, then older values of cycle_ctr might remain when we + * come back here to try again. Repeated checkpoint failures would + * eventually wrap the counter around to the point where an old entry + * might appear new, causing us to skip it, possibly allowing a checkpoint + * to succeed that should not have. To forestall wraparound, any time the + * previous ProcessFsyncRequests() failed to complete, run through the + * table and forcibly set cycle_ctr = sync_cycle_ctr. + * + * Think not to merge this loop with the main loop, as the problem is + * exactly that that loop may fail before having visited all the entries. + * From a performance point of view it doesn't matter anyway, as this path + * will never be taken in a system that's functioning normally. + */ + if (sync_in_progress) + { + /* prior try failed, so update any stale cycle_ctr values */ + hash_seq_init(&hstat, pendingOps); + while ((entry = (PendingFsyncEntry *) hash_seq_search(&hstat)) != NULL) + { + entry->cycle_ctr = sync_cycle_ctr; + } + } + + /* Advance counter so that new hashtable entries are distinguishable */ + sync_cycle_ctr++; + + /* Set flag to detect failure if we don't reach the end of the loop */ + sync_in_progress = true; + + /* Now scan the hashtable for fsync requests to process */ + absorb_counter = FSYNCS_PER_ABSORB; + hash_seq_init(&hstat, pendingOps); + while ((entry = (PendingFsyncEntry *) hash_seq_search(&hstat)) != NULL) + { + int failures; + + /* + * If fsync is off then we don't have to bother opening the + * file at all. (We delay checking until this point so that + * changing fsync on the fly behaves sensibly.) + */ + if (!enableFsync) + continue; + + /* + * If the entry is new then don't process it this time; it might + * contain multiple fsync-request bits, but they are all new. Note + * "continue" bypasses the hash-remove call at the bottom of the loop. + */ + if (entry->cycle_ctr == sync_cycle_ctr) + continue; + + /* Else assert we haven't missed it */ + Assert((CycleCtr) (entry->cycle_ctr + 1) == sync_cycle_ctr); + + /* + * If in checkpointer, we want to absorb pending requests + * every so often to prevent overflow of the fsync request + * queue. It is unspecified whether newly-added entries will + * be visited by hash_seq_search, but we don't care since we + * don't need to process them anyway. + */ + if (--absorb_counter <= 0) + { + AbsorbSyncRequests(); + absorb_counter = FSYNCS_PER_ABSORB; + } + + /* + * The fsync table could contain requests to fsync segments + * that have been deleted (unlinked) by the time we get to + * them. Rather than just hoping an ENOENT (or EACCES on + * Windows) error can be ignored, what we do on error is + * absorb pending requests and then retry. Since mdunlink() + * queues a "cancel" message before actually unlinking, the + * fsync request is guaranteed to be marked canceled after the + * absorb if it really was this case. DROP DATABASE likewise + * has to tell us to forget fsync requests before it starts + * deletions. + * + * If the entry was cancelled after the absorb above, or within the + * absorb inside the loop, exit the loop. We delete the entry right + * after. Look can also exit at "break". + */ + for (failures = 0; !(entry->canceled); failures++) + { + char *path; + File fd; + + path = syncsw[entry->handler].sync_filepath(&(entry->ftag)); + fd = PathNameOpenFile(path, O_RDWR | PG_BINARY); + + INSTR_TIME_SET_CURRENT(sync_start); + if (fd >= 0 && + FileSync(fd, WAIT_EVENT_DATA_FILE_SYNC) >= 0) + { + /* Success; update statistics about sync timing */ + INSTR_TIME_SET_CURRENT(sync_end); + sync_diff = sync_end; + INSTR_TIME_SUBTRACT(sync_diff, sync_start); + elapsed = INSTR_TIME_GET_MICROSEC(sync_diff); + if (elapsed > longest) + longest = elapsed; + total_elapsed += elapsed; + processed++; + + if (log_checkpoints) + elog(DEBUG1, "checkpoint sync: number=%d file=%s time=%.3f msec", + processed, + path, + (double) elapsed / 1000); + + FileClose(fd); + pfree(path); + break; /* out of retry loop */ + } + + /* Done with the file descriptor, close it */ + if (fd >= 0) + FileClose(fd); + + /* + * It is possible that the relation has been dropped or + * truncated since the fsync request was entered. + * Therefore, allow ENOENT, but only if we didn't fail + * already on this file. This applies both for + * smgrgetseg() and for FileSync, since fd.c might have + * closed the file behind our back. + * + * XXX is there any point in allowing more than one retry? + * Don't see one at the moment, but easy to change the + * test here if so. + */ + if (!FILE_POSSIBLY_DELETED(errno) || failures > 0) + ereport(data_sync_elevel(ERROR), + (errcode_for_file_access(), + errmsg("could not fsync file \"%s\": %m", + path))); + else + ereport(DEBUG1, + (errcode_for_file_access(), + errmsg("could not fsync file \"%s\" but retrying: %m", + path))); + + pfree(path); + + /* + * Absorb incoming requests and check to see if a cancel + * arrived for this relation fork. + */ + AbsorbSyncRequests(); + absorb_counter = FSYNCS_PER_ABSORB; /* might as well... */ + } /* end retry loop */ + + /* We are done with this entry, remove it */ + if (hash_search(pendingOps, &entry->ftag, HASH_REMOVE, NULL) == NULL) + elog(ERROR, "pendingOps corrupted"); + } /* end loop over hashtable entries */ + + /* Return sync performance metrics for report at checkpoint end */ + CheckpointStats.ckpt_sync_rels = processed; + CheckpointStats.ckpt_longest_sync = longest; + CheckpointStats.ckpt_agg_sync_time = total_elapsed; + + /* Flag successful completion of ProcessSyncRequests */ + sync_in_progress = false; +} + +/* + * RememberSyncRequest() -- callback from checkpointer side of sync request + * + * We stuff fsync requests into the local hash table for execution + * during the checkpointer's next checkpoint. UNLINK requests go into a + * separate linked list, however, because they get processed separately. + * + * See sync.h for more information on the types of sync requests supported. + */ +void +RememberSyncRequest(const FileTag *ftag, SyncRequestType type, SyncRequestHandler handler) +{ + Assert(pendingOps); + + if (type == SYNC_FORGET_REQUEST) + { + PendingFsyncEntry *entry; + /* Cancel previously entered request */ + entry = (PendingFsyncEntry *) hash_search(pendingOps, + (void *)ftag, + HASH_FIND, + NULL); + if (entry != NULL) + entry->canceled = true; + } + else if (type == SYNC_FORGET_HIERARCHY_REQUEST) + { + /* Remove any pending requests for the entire database */ + HASH_SEQ_STATUS hstat; + PendingFsyncEntry *entry; + ListCell *cell, + *prev, + *next; + + /* Remove fsync requests */ + hash_seq_init(&hstat, pendingOps); + while ((entry = (PendingFsyncEntry *) hash_seq_search(&hstat)) != NULL) + { + if (syncsw[entry->handler].sync_filetagmatches(&(entry->ftag), + ftag /* predicate */, type)) + { + entry->canceled = true; + } + } + + /* Remove unlink requests */ + prev = NULL; + for (cell = list_head(pendingUnlinks); cell; cell = next) + { + PendingUnlinkEntry *entry = (PendingUnlinkEntry *) lfirst(cell); + + next = lnext(cell); + if (syncsw[entry->handler].sync_filetagmatches(&(entry->ftag), + ftag /* predicate */, type)) + { + pendingUnlinks = list_delete_cell(pendingUnlinks, cell, prev); + pfree(entry); + } + else + prev = cell; + } + } + else if (type == SYNC_UNLINK_REQUEST) + { + /* Unlink request: put it in the linked list */ + MemoryContext oldcxt = MemoryContextSwitchTo(pendingOpsCxt); + PendingUnlinkEntry *entry; + + entry = palloc(sizeof(PendingUnlinkEntry)); + entry->ftag = *ftag; + entry->handler = handler; + entry->cycle_ctr = checkpoint_cycle_ctr; + + pendingUnlinks = lappend(pendingUnlinks, entry); + + MemoryContextSwitchTo(oldcxt); + } + else + { + /* Normal case: enter a request to fsync this segment */ + MemoryContext oldcxt = MemoryContextSwitchTo(pendingOpsCxt); + PendingFsyncEntry *entry; + bool found; + + Assert(type == SYNC_REQUEST); + + entry = (PendingFsyncEntry *) hash_search(pendingOps, + ftag, + HASH_ENTER, + &found); + /* if new entry, initialize it */ + if (!found) + { + entry->handler = handler; + entry->cycle_ctr = sync_cycle_ctr; + entry->canceled = false; + } + + /* + * NB: it's intentional that we don't change cycle_ctr if the entry + * already exists. The cycle_ctr must represent the oldest fsync + * request that could be in the entry. + */ + + MemoryContextSwitchTo(oldcxt); + } +} + +/* + * RegisterSyncRequest() + * + * Register the sync request locally, or forward it to the checkpointer. + * Caller can chose to infinitely retry or return immediately on error. We + * currently will wait for 10 ms before retrying. + */ +bool +RegisterSyncRequest(const FileTag *ftag, SyncRequestType type, + SyncRequestHandler handler, bool retryOnError) +{ + bool ret; + + if (pendingOps != NULL) + { + /* standalone backend or startup process: fsync state is local */ + RememberSyncRequest(ftag, type, handler); + return true; + } + else + { + while(1) + { + /* + * Notify the checkpointer about it. If we fail to queue the cancel + * message, we have to sleep and try again ... ugly, but hopefully + * won't happen often. + * + * XXX should we CHECK_FOR_INTERRUPTS in this loop? Escaping with an + * error would leave the no-longer-used file still present on disk, + * which would be bad, so I'm inclined to assume that the checkpointer + * will always empty the queue soon. + */ + ret = ForwardSyncRequest(ftag, type, handler); + + /* + * If we are successful in queueing the request, or we failed and + * was instructed not to retry on error, break. + */ + if (ret || (!ret && !retryOnError)) + break; + + pg_usleep(10000L); + } + + return ret; + } +} + +/* + * In archive recovery, we rely on checkpointer to do fsyncs, but we will have + * already created the pendingOps during initialization of the startup + * process. Calling this function drops the local pendingOps so that + * subsequent requests will be forwarded to checkpointer. + */ +void +EnableSyncRequestForwarding(void) +{ + /* Perform any pending fsyncs we may have queued up, then drop table */ + if (pendingOps) + { + ProcessSyncRequests(); + hash_destroy(pendingOps); + } + pendingOps = NULL; + + /* + * We should not have any pending unlink requests, since mdunlink doesn't + * queue unlink requests when isRedo. + */ + Assert(pendingUnlinks == NIL); +} diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c index a5ee209f91..0326e6c6ed 100644 --- a/src/backend/utils/init/postinit.c +++ b/src/backend/utils/init/postinit.c @@ -50,6 +50,7 @@ #include "storage/proc.h" #include "storage/sinvaladt.h" #include "storage/smgr.h" +#include "storage/sync.h" #include "tcop/tcopprot.h" #include "utils/acl.h" #include "utils/fmgroids.h" @@ -554,6 +555,7 @@ BaseInit(void) /* Do local initialization of file, storage and buffer managers */ InitFileAccess(); + InitSync(); smgrinit(); InitBufferPoolAccess(); } diff --git a/src/include/postmaster/bgwriter.h b/src/include/postmaster/bgwriter.h index 53b8f5fe3c..40b05d4661 100644 --- a/src/include/postmaster/bgwriter.h +++ b/src/include/postmaster/bgwriter.h @@ -17,6 +17,8 @@ #include "storage/block.h" #include "storage/relfilenode.h" +#include "storage/smgr.h" +#include "storage/sync.h" /* GUC options */ @@ -31,9 +33,9 @@ extern void CheckpointerMain(void) pg_attribute_noreturn(); extern void RequestCheckpoint(int flags); extern void CheckpointWriteDelay(int flags, double progress); -extern bool ForwardFsyncRequest(RelFileNode rnode, ForkNumber forknum, - BlockNumber segno); -extern void AbsorbFsyncRequests(void); +extern bool ForwardSyncRequest(const FileTag *ftag, SyncRequestType type, + SyncRequestHandler handler); +extern void AbsorbSyncRequests(void); extern Size CheckpointerShmemSize(void); extern void CheckpointerShmemInit(void); diff --git a/src/include/storage/fd.h b/src/include/storage/fd.h index 74c34757fb..40f46b871d 100644 --- a/src/include/storage/fd.h +++ b/src/include/storage/fd.h @@ -54,6 +54,18 @@ extern PGDLLIMPORT bool data_sync_retry; */ extern int max_safe_fds; +/* + * On Windows, we have to interpret EACCES as possibly meaning the same as + * ENOENT, because if a file is unlinked-but-not-yet-gone on that platform, + * that's what you get. Ugh. This code is designed so that we don't + * actually believe these cases are okay without further evidence (namely, + * a pending fsync request getting canceled ... see mdsync). + */ +#ifndef WIN32 +#define FILE_POSSIBLY_DELETED(err) ((err) == ENOENT) +#else +#define FILE_POSSIBLY_DELETED(err) ((err) == ENOENT || (err) == EACCES) +#endif /* * prototypes for functions in fd.c diff --git a/src/include/storage/md.h b/src/include/storage/md.h new file mode 100644 index 0000000000..fc13e34a6f --- /dev/null +++ b/src/include/storage/md.h @@ -0,0 +1,51 @@ +/*------------------------------------------------------------------------- + * + * md.h + * magnetic disk storage manager public interface declarations. + * + * + * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * src/include/storage/md.h + * + *------------------------------------------------------------------------- + */ +#ifndef MD_H +#define MD_H + +#include "storage/block.h" +#include "storage/relfilenode.h" +#include "storage/smgr.h" +#include "storage/sync.h" + +/* md storage manager functionality */ +extern void mdinit(void); +extern void mdclose(SMgrRelation reln, ForkNumber forknum); +extern void mdcreate(SMgrRelation reln, ForkNumber forknum, bool isRedo); +extern bool mdexists(SMgrRelation reln, ForkNumber forknum); +extern void mdunlink(RelFileNodeBackend rnode, ForkNumber forknum, bool isRedo); +extern void mdextend(SMgrRelation reln, ForkNumber forknum, + BlockNumber blocknum, char *buffer, bool skipFsync); +extern void mdprefetch(SMgrRelation reln, ForkNumber forknum, + BlockNumber blocknum); +extern void mdread(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, + char *buffer); +extern void mdwrite(SMgrRelation reln, ForkNumber forknum, + BlockNumber blocknum, char *buffer, bool skipFsync); +extern void mdwriteback(SMgrRelation reln, ForkNumber forknum, + BlockNumber blocknum, BlockNumber nblocks); +extern BlockNumber mdnblocks(SMgrRelation reln, ForkNumber forknum); +extern void mdtruncate(SMgrRelation reln, ForkNumber forknum, + BlockNumber nblocks); +extern void mdimmedsync(SMgrRelation reln, ForkNumber forknum); + +extern void ForgetDatabaseSyncRequests(Oid dbid); +extern void DropRelationFiles(RelFileNode *delrels, int ndelrels, bool isRedo); + +/* md sync callbacks */ +extern char *mdfilepath(const FileTag *ftag); +extern bool mdfiletagmatches(const FileTag *ftag, const FileTag *predicate, + SyncRequestType type); + +#endif /* MD_H */ diff --git a/src/include/storage/segment.h b/src/include/storage/segment.h new file mode 100644 index 0000000000..c7af945168 --- /dev/null +++ b/src/include/storage/segment.h @@ -0,0 +1,28 @@ +/*------------------------------------------------------------------------- + * + * segment.h + * POSTGRES disk segment definitions. + * + * + * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * src/include/storage/segment.h + * + *------------------------------------------------------------------------- + */ +#ifndef SEGMENT_H +#define SEGMENT_H + + +/* + * Segment Number: + * + * Each relation and its forks are divided into segments. This + * definition formalizes the definition of the segment number. + */ +typedef uint32 SegmentNumber; + +#define InvalidSegmentNumber ((SegmentNumber) 0xFFFFFFFF) + +#endif /* SEGMENT_H */ diff --git a/src/include/storage/smgr.h b/src/include/storage/smgr.h index 820d08ed4e..26ac8f2cec 100644 --- a/src/include/storage/smgr.h +++ b/src/include/storage/smgr.h @@ -18,7 +18,6 @@ #include "storage/block.h" #include "storage/relfilenode.h" - /* * smgr.c maintains a table of SMgrRelation objects, which are essentially * cached file handles. An SMgrRelation is created (if not already present) @@ -106,43 +105,6 @@ extern BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum); extern void smgrtruncate(SMgrRelation reln, ForkNumber forknum, BlockNumber nblocks); extern void smgrimmedsync(SMgrRelation reln, ForkNumber forknum); -extern void smgrpreckpt(void); -extern void smgrsync(void); -extern void smgrpostckpt(void); extern void AtEOXact_SMgr(void); - -/* internals: move me elsewhere -- ay 7/94 */ - -/* in md.c */ -extern void mdinit(void); -extern void mdclose(SMgrRelation reln, ForkNumber forknum); -extern void mdcreate(SMgrRelation reln, ForkNumber forknum, bool isRedo); -extern bool mdexists(SMgrRelation reln, ForkNumber forknum); -extern void mdunlink(RelFileNodeBackend rnode, ForkNumber forknum, bool isRedo); -extern void mdextend(SMgrRelation reln, ForkNumber forknum, - BlockNumber blocknum, char *buffer, bool skipFsync); -extern void mdprefetch(SMgrRelation reln, ForkNumber forknum, - BlockNumber blocknum); -extern void mdread(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, - char *buffer); -extern void mdwrite(SMgrRelation reln, ForkNumber forknum, - BlockNumber blocknum, char *buffer, bool skipFsync); -extern void mdwriteback(SMgrRelation reln, ForkNumber forknum, - BlockNumber blocknum, BlockNumber nblocks); -extern BlockNumber mdnblocks(SMgrRelation reln, ForkNumber forknum); -extern void mdtruncate(SMgrRelation reln, ForkNumber forknum, - BlockNumber nblocks); -extern void mdimmedsync(SMgrRelation reln, ForkNumber forknum); -extern void mdpreckpt(void); -extern void mdsync(void); -extern void mdpostckpt(void); - -extern void SetForwardFsyncRequests(void); -extern void RememberFsyncRequest(RelFileNode rnode, ForkNumber forknum, - BlockNumber segno); -extern void ForgetRelationFsyncRequests(RelFileNode rnode, ForkNumber forknum); -extern void ForgetDatabaseFsyncRequests(Oid dbid); -extern void DropRelationFiles(RelFileNode *delrels, int ndelrels, bool isRedo); - #endif /* SMGR_H */ diff --git a/src/include/storage/sync.h b/src/include/storage/sync.h new file mode 100644 index 0000000000..11a0f01d42 --- /dev/null +++ b/src/include/storage/sync.h @@ -0,0 +1,86 @@ +/*------------------------------------------------------------------------- + * + * sync.h + * File synchronization management code. + * + * Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * src/include/storage/sync.h + * + *------------------------------------------------------------------------- + */ +#ifndef SYNC_H +#define SYNC_H + +#include "storage/block.h" +#include "storage/relfilenode.h" +#include "storage/segment.h" + +/* + * Caller specified type of sync request. + * + * SYNC_REQUESTs are issued to sync a particular file whose path is determined + * by calling back the handler. A SYNC_FORGET_REQUEST instructs the sync + * mechanism to cancel a previously submitted sync request. + * + * SYNC_FORGET_HIERARCHY_REQUEST is a special type of forget request that + * involves scanning all pending sync requests and cancelling any entry that + * matches. The entries are resolved by calling back the handler as the key is + * opaque to the sync mechanism. Handling these types of requests are a tad slow + * because we have to search all the requests linearly, but usage of this such + * as dropping databases, is a pretty heavyweight operation anyhow, so we'll + * live with it. + * + * SYNC_UNLINK_REQUEST is a request to delete the file after the next + * checkpoint. The path is determined by calling back the handler. + */ +typedef enum syncrequesttype +{ + SYNC_REQUEST, + SYNC_FORGET_REQUEST, + SYNC_FORGET_HIERARCHY_REQUEST, + SYNC_UNLINK_REQUEST +} SyncRequestType; + +/* + * Identifies the handler for the sync callbacks. + * + * These enums map back to entries in the callback function table. For + * consistency, explicitly set the value to 0. See sync.c for more information. + */ +typedef enum syncrequesthandler +{ + SYNC_HANDLER_MD = 0 /* md smgr */ +} SyncRequestHandler; + +/* + * Augmenting a relfilenode with the fork and segment number provides all + * the information to locate the particular segment of interest for a relation. + */ +typedef struct filetag +{ + RelFileNode rnode; + ForkNumber forknum; + SegmentNumber segno; +} FileTag; + +#define INIT_FILETAG(a,xx_rnode,xx_forknum,xx_segno) \ +( \ + (a).rnode = (xx_rnode), \ + (a).forknum = (xx_forknum), \ + (a).segno = (xx_segno) \ +) + +/* sync forward declarations */ +extern void InitSync(void); +extern void SyncPreCheckpoint(void); +extern void SyncPostCheckpoint(void); +extern void ProcessSyncRequests(void); +extern void RememberSyncRequest(const FileTag *ftag, SyncRequestType type, + SyncRequestHandler handler); +extern void EnableSyncRequestForwarding(void); +extern bool RegisterSyncRequest(const FileTag *ftag, SyncRequestType type, + SyncRequestHandler handler, bool retryOnError); + +#endif /* SYNC_H */ -- 2.16.5