aio: WAL: WIP: concurrent WAL flushes
authorAndres Freund <[email protected]>
Mon, 11 Jan 2021 21:52:06 +0000 (13:52 -0800)
committerAndres Freund <[email protected]>
Mon, 11 Jan 2021 23:09:15 +0000 (15:09 -0800)
src/backend/access/transam/xlog.c
src/backend/storage/buffer/bufmgr.c
src/backend/storage/ipc/aio.c
src/backend/storage/lmgr/lwlocknames.txt
src/backend/utils/misc/guc.c
src/include/storage/bufmgr.h
src/include/storage/proc.h

index 1d168ed9c80d1c956d2fff2fb98747a6e7fe49ba..56d1d29fbe6aed6ace0534dc5b4581d6792f4f91 100644 (file)
@@ -48,6 +48,7 @@
 #include "pg_trace.h"
 #include "pgstat.h"
 #include "port/atomics.h"
+#include "port/pg_bitutils.h"
 #include "port/pg_iovec.h"
 #include "postmaster/bgwriter.h"
 #include "postmaster/startup.h"
@@ -69,6 +70,7 @@
 #include "storage/predicate.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
+#include "storage/proclist.h"
 #include "storage/reinit.h"
 #include "storage/smgr.h"
 #include "storage/spin.h"
@@ -440,14 +442,18 @@ static XLogRecPtr RedoStartLSN = InvalidXLogRecPtr;
 
 typedef struct XLogwrtRqst
 {
-   XLogRecPtr  Write;          /* last byte + 1 to write out */
-   XLogRecPtr  Flush;          /* last byte + 1 to flush */
+   XLogRecPtr  WriteInit;      /* last byte + 1 to start to write out */
+   XLogRecPtr  WriteDone;      /* last byte + 1 to be done writing out */
+   XLogRecPtr  FlushInit;      /* last byte + 1 to start flushing */
+   XLogRecPtr  FlushDone;      /* last byte + 1 to be done flushing */
 } XLogwrtRqst;
 
 typedef struct XLogwrtResult
 {
-   XLogRecPtr  Write;          /* last byte + 1 written out */
-   XLogRecPtr  Flush;          /* last byte + 1 flushed */
+   XLogRecPtr  WriteInit;      /* last byte + 1 being written out */
+   XLogRecPtr  WriteDone;      /* last byte + 1 done writing out */
+   XLogRecPtr  FlushInit;      /* last byte + 1 being flushed */
+   XLogRecPtr  FlushDone;      /* last byte + 1 done flushing */
 } XLogwrtResult;
 
 /*
@@ -596,6 +602,90 @@ typedef struct XLogCtlInsert
    WALInsertLockPadded *WALInsertLocks;
 } XLogCtlInsert;
 
+typedef struct XLBlockPageStatus
+{
+   XLogRecPtr lsn;     /* 1st byte ptr-s + XLOG_BLCKSZ */
+   uint32 io;
+} XLBlockPageStatus;
+
+typedef struct XLogIO
+{
+   XLogRecPtr upto; /* covers LSNs <= */
+   bool in_progress;
+   PgAioIoRef aio_ref;
+} XLogIO;
+
+/*
+ * Simple ringbuffer for managing in-flight IOs.
+ *
+ * Indices are *not* wrapped by num_ios, but wrap at integer range.
+ *
+ * Empty: tail == next
+ * Size: (uint16)(next - tail)
+ * Full: Size == num_ios
+ *
+
+ * This is done so both full and empty queues can be represented well.
+ */
+typedef struct XLogIOQueue
+{
+   uint16 mask;
+   uint16 num_ios;
+   uint16 tail;
+   uint16 next;
+   XLogIO ios[];
+} XLogIOQueue;
+
+static size_t
+XLogIOQueueMem(void)
+{
+   size_t sz;
+
+   sz = sizeof(XLogIOQueue);
+   sz =
+       add_size(sz,
+                mul_size(mul_size(sizeof(XLogIO), io_wal_concurrency),
+                         2));
+
+   return sz;
+}
+
+static inline bool
+XLogIOQueueEmpty(XLogIOQueue *queue)
+{
+   return queue->tail == queue->next;
+}
+
+static inline uint16
+XLogIOQueueSize(XLogIOQueue *queue)
+{
+   return (uint16)(queue->next - queue->tail);
+}
+
+static inline bool
+XLogIOQueueFull(XLogIOQueue *queue)
+{
+   return XLogIOQueueSize(queue) == queue->num_ios;
+}
+
+static inline XLogIO*
+XLogIOQueueElem(XLogIOQueue *queue, uint16 elem)
+{
+   return &queue->ios[elem & queue->mask];
+}
+
+static inline XLogIO*
+XLogIOQueueHead(XLogIOQueue *queue)
+{
+   return XLogIOQueueElem(queue, queue->next - 1);
+}
+
+static inline XLogIO*
+XLogIOQueueTail(XLogIOQueue *queue)
+{
+   return XLogIOQueueElem(queue, queue->tail);
+}
+
 /*
  * Total shared-memory state for XLOG.
  */
@@ -626,6 +716,9 @@ typedef struct XLogCtlData
     */
    XLogwrtResult LogwrtResult;
 
+   XLogIOQueue *writes;
+   XLogIOQueue *flushes;
+
    /*
     * Latest initialized page in the cache (last byte position + 1).
     *
@@ -644,9 +737,14 @@ typedef struct XLogCtlData
     * WALBufMappingLock.
     */
    char       *pages;          /* buffers for unwritten XLOG pages */
-   XLogRecPtr *xlblocks;       /* 1st byte ptr-s + XLOG_BLCKSZ */
+   XLBlockPageStatus *xlblocks;        /* 1st byte ptr-s + XLOG_BLCKSZ */
    int         XLogCacheBlck;  /* highest allocated xlog buffer index */
 
+   /*
+    * One reusable buffer for zeroing out WAL files.
+    */
+   char       *zerobuf;
+
    /*
     * Shared copy of ThisTimeLineID. Does not change after end-of-recovery.
     * If we created a new timeline when the system was started up,
@@ -917,7 +1015,7 @@ static XLogRecPtr XLogGetReplicationSlotMinimumLSN(void);
 
 static void AdvanceXLInsertBuffer(XLogRecPtr upto, bool opportunistic);
 static bool XLogCheckpointNeeded(XLogSegNo new_segno);
-static void XLogWrite(XLogwrtRqst WriteRqst, bool flexible);
+static bool XLogWrite(XLogwrtRqst WriteRqst, bool flexible);
 static bool InstallXLogFileSegment(XLogSegNo *segno, char *tmppath,
                                   bool find_free, XLogSegNo max_segno,
                                   bool use_lock);
@@ -984,6 +1082,7 @@ static void WALInsertLockAcquireExclusive(void);
 static void WALInsertLockRelease(void);
 static void WALInsertLockUpdateInsertingAt(XLogRecPtr insertingAt);
 
+
 /*
  * Insert an XLOG record represented by an already-constructed chain of data
  * chunks.  This is a low-level routine; to construct the WAL record header
@@ -1176,8 +1275,13 @@ XLogInsertRecord(XLogRecData *rdata,
    {
        SpinLockAcquire(&XLogCtl->info_lck);
        /* advance global request to include new block(s) */
-       if (XLogCtl->LogwrtRqst.Write < EndPos)
-           XLogCtl->LogwrtRqst.Write = EndPos;
+       if (XLogCtl->LogwrtRqst.WriteInit < EndPos)
+           XLogCtl->LogwrtRqst.WriteInit = EndPos;
+#if 0
+       if (XLogCtl->LogwrtRqst.WriteDone < EndPos)
+           XLogCtl->LogwrtRqst.WriteDone = EndPos;
+#endif
+
        /* update local result copy while I have the chance */
        LogwrtResult = XLogCtl->LogwrtResult;
        SpinLockRelease(&XLogCtl->info_lck);
@@ -1977,7 +2081,7 @@ GetXLogBuffer(XLogRecPtr ptr)
    expectedEndPtr = ptr;
    expectedEndPtr += XLOG_BLCKSZ - ptr % XLOG_BLCKSZ;
 
-   endptr = XLogCtl->xlblocks[idx];
+   endptr = XLogCtl->xlblocks[idx].lsn;
    if (expectedEndPtr != endptr)
    {
        XLogRecPtr  initializedUpto;
@@ -2008,7 +2112,7 @@ GetXLogBuffer(XLogRecPtr ptr)
        WALInsertLockUpdateInsertingAt(initializedUpto);
 
        AdvanceXLInsertBuffer(ptr, false);
-       endptr = XLogCtl->xlblocks[idx];
+       endptr = XLogCtl->xlblocks[idx].lsn;
 
        if (expectedEndPtr != endptr)
            elog(PANIC, "could not find WAL buffer for %X/%X",
@@ -2195,8 +2299,8 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, bool opportunistic)
         * be zero if the buffer hasn't been used yet).  Fall through if it's
         * already written out.
         */
-       OldPageRqstPtr = XLogCtl->xlblocks[nextidx];
-       if (LogwrtResult.Write < OldPageRqstPtr)
+       OldPageRqstPtr = XLogCtl->xlblocks[nextidx].lsn;
+       if (LogwrtResult.WriteDone < OldPageRqstPtr)
        {
            /*
             * Nope, got work to do. If we just want to pre-initialize as much
@@ -2207,8 +2311,8 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, bool opportunistic)
 
            /* Before waiting, get info_lck and update LogwrtResult */
            SpinLockAcquire(&XLogCtl->info_lck);
-           if (XLogCtl->LogwrtRqst.Write < OldPageRqstPtr)
-               XLogCtl->LogwrtRqst.Write = OldPageRqstPtr;
+           if (XLogCtl->LogwrtRqst.WriteDone < OldPageRqstPtr)
+               XLogCtl->LogwrtRqst.WriteDone = OldPageRqstPtr;
            LogwrtResult = XLogCtl->LogwrtResult;
            SpinLockRelease(&XLogCtl->info_lck);
 
@@ -2216,7 +2320,7 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, bool opportunistic)
             * Now that we have an up-to-date LogwrtResult value, see if we
             * still need to write it or if someone else already did.
             */
-           if (LogwrtResult.Write < OldPageRqstPtr)
+           if (LogwrtResult.WriteDone < OldPageRqstPtr)
            {
                /*
                 * Must acquire write lock. Release WALBufMappingLock first,
@@ -2226,31 +2330,17 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, bool opportunistic)
                 */
                LWLockRelease(WALBufMappingLock);
 
-               if (!XLogInsertionsKnownFinished(OldPageRqstPtr))
-               {
-                   WaitXLogInsertionsToFinish(OldPageRqstPtr);
-                   Assert(XLogInsertionsKnownFinished(OldPageRqstPtr));
-               }
-
-               LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
+               /* Have to write it ourselves */
+               // FIXME: trace event wrong
+               TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_START();
+               WriteRqst.WriteInit = 0;
+               WriteRqst.WriteDone = OldPageRqstPtr;
+               WriteRqst.FlushInit = 0;
+               WriteRqst.FlushDone = 0;
+               XLogWrite(WriteRqst, false);
+               pgWalUsage.wal_buffers_full++;
+               TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 
-               LogwrtResult = XLogCtl->LogwrtResult;
-               if (LogwrtResult.Write >= OldPageRqstPtr)
-               {
-                   /* OK, someone wrote it already */
-                   LWLockRelease(WALWriteLock);
-               }
-               else
-               {
-                   /* Have to write it ourselves */
-                   TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_START();
-                   WriteRqst.Write = OldPageRqstPtr;
-                   WriteRqst.Flush = 0;
-                   XLogWrite(WriteRqst, false);
-                   LWLockRelease(WALWriteLock);
-                   pgWalUsage.wal_buffers_full++;
-                   TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
-               }
                /* Re-acquire WALBufMappingLock and retry */
                LWLockAcquire(WALBufMappingLock, LW_EXCLUSIVE);
                continue;
@@ -2321,7 +2411,7 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, bool opportunistic)
         */
        pg_write_barrier();
 
-       *((volatile XLogRecPtr *) &XLogCtl->xlblocks[nextidx]) = NewPageEndPtr;
+       *((volatile XLogRecPtr *) &XLogCtl->xlblocks[nextidx].lsn) = NewPageEndPtr;
 
        XLogCtl->InitializedUpTo = NewPageEndPtr;
 
@@ -2451,31 +2541,416 @@ XLogCheckpointNeeded(XLogSegNo new_segno)
    return false;
 }
 
+static void
+XLogIOQueueCheck(XLogIOQueue *queue)
+{
+#if USE_ASSERT_CHECKING
+   XLogRecPtr upto = 0;
+
+   Assert(LWLockHeldByMe(WALIOQueueLock));
+
+   for (uint16 cur = queue->tail; cur != queue->next; cur++)
+   {
+       XLogIO *curio = XLogIOQueueElem(queue, cur);
+
+       Assert(curio->upto != 0);
+       Assert(upto < curio->upto);
+       upto = curio->upto;
+   }
+#endif
+}
 
 void
 XLogWriteComplete(PgAioInProgress *aio, uint32 write_no)
 {
+   XLogIOQueue *writes = XLogCtl->writes;
+   XLogIO *io;
+
+   LWLockAcquire(WALIOQueueLock, LW_EXCLUSIVE);
+
+   Assert(write_no == (write_no & writes->mask));
+   io = &writes->ios[write_no];
+
+   Assert(io->in_progress);
+   io->in_progress = false;
+   pg_write_barrier();
+   pgaio_io_ref_clear(&io->aio_ref);
+
+   Assert(!XLogIOQueueEmpty(writes));
+
+   if (write_no == (writes->tail & writes->mask))
+   {
+       XLogRecPtr upto = 0;
+       uint16 new_tail = 0;
+
+       for (uint16 cur = writes->tail; cur != writes->next; cur++)
+       {
+           XLogIO *curio = XLogIOQueueElem(writes, cur);
+
+           if (curio->in_progress)
+               break;
+
+           Assert(upto < curio->upto);
+           upto = curio->upto;
+
+           Assert(upto != 0);
+           curio->upto = 0;
+           new_tail = cur + 1;
+       }
+
+       if (0)
+       {
+           elog(DEBUG1, "updating write queue tail from %d to %d, lsn %X/%X",
+                writes->tail, new_tail,
+                (uint32)(upto >> 32), (uint32) upto);
+       }
+
+       Assert(writes->tail != new_tail);
+       writes->tail = new_tail;
+
+       Assert(upto != 0);
+
+       SpinLockAcquire(&XLogCtl->info_lck);
+
+       Assert(upto <= XLogCtl->LogwrtResult.WriteInit);
+       Assert(upto > XLogCtl->LogwrtResult.WriteDone);
+
+       if (upto == 0)
+           elog(PANIC, "WTF");
+
+       XLogCtl->LogwrtResult.WriteDone = upto;
+
+       if (upto > XLogCtl->LogwrtRqst.WriteDone)
+           XLogCtl->LogwrtRqst.WriteDone = upto;
+
+       if (sync_method == SYNC_METHOD_OPEN ||
+           sync_method == SYNC_METHOD_OPEN_DSYNC)
+       {
+           Assert(upto <= XLogCtl->LogwrtResult.FlushInit);
+           Assert(upto > XLogCtl->LogwrtResult.FlushDone);
+
+           XLogCtl->LogwrtResult.FlushDone = upto;
+           if (upto > XLogCtl->LogwrtRqst.FlushDone)
+               XLogCtl->LogwrtRqst.FlushDone = upto;
+       }
+
+       SpinLockRelease(&XLogCtl->info_lck);
+   }
+
+   XLogIOQueueCheck(writes);
+
+   LWLockRelease(WALIOQueueLock);
+
 }
 
 void
 XLogFlushComplete(struct PgAioInProgress *aio, uint32 flush_no)
 {
+   XLogIOQueue *flushes = XLogCtl->flushes;
+   XLogIO *io;
+
+   LWLockAcquire(WALIOQueueLock, LW_EXCLUSIVE);
+
+   Assert(sync_method != SYNC_METHOD_OPEN &&
+          sync_method != SYNC_METHOD_OPEN_DSYNC);
+
+   Assert(flush_no == (flush_no & flushes->mask));
+   io = &flushes->ios[flush_no];
+
+   Assert(io->in_progress);
+   io->in_progress = false;
+   pg_write_barrier();
+   pgaio_io_ref_clear(&io->aio_ref);
+
+   if (flush_no == (flushes->tail & flushes->mask))
+   {
+       XLogRecPtr upto = 0;
+       uint16 new_tail = 0;
+
+       for (uint16 cur = flushes->tail; cur != flushes->next; cur++)
+       {
+           XLogIO *curio = XLogIOQueueElem(flushes, cur);
+
+           if (curio->in_progress)
+               break;
+
+           Assert(upto < curio->upto);
+           upto = curio->upto;
+
+           Assert(upto != 0);
+           curio->upto = 0;
+           new_tail = cur + 1;
+       }
+
+       if (0)
+       {
+           elog(DEBUG3, "updating flush queue tail from %d to %d, lsn %X/%X",
+                flushes->tail, new_tail,
+                (uint32)(upto >> 32), (uint32) upto);
+       }
+
+       Assert(flushes->tail != new_tail);
+       flushes->tail = new_tail;
+
+       Assert(upto != 0);
+
+       SpinLockAcquire(&XLogCtl->info_lck);
+
+       Assert(upto <= XLogCtl->LogwrtResult.FlushInit);
+       Assert(upto >= XLogCtl->LogwrtResult.FlushDone);
+
+       if (upto > XLogCtl->LogwrtRqst.FlushDone)
+           XLogCtl->LogwrtRqst.FlushDone = upto;
+
+       if (upto > XLogCtl->LogwrtResult.FlushDone)
+           XLogCtl->LogwrtResult.FlushDone = upto;
+
+       LogwrtResult = XLogCtl->LogwrtResult;
+       SpinLockRelease(&XLogCtl->info_lck);
+   }
+
+   LWLockRelease(WALIOQueueLock);
+}
+
+static bool
+XLogIOQueueWaitFor(XLogIOQueue *queue, XLogRecPtr lsn, bool release_write_lock)
+{
+   PgAioIoRef aio_refs[128];
+   int waitcount = 0;
+   bool released_lock = false;
+
+   Assert(!LWLockHeldByMe(WALIOQueueLock));
+   LWLockAcquire(WALIOQueueLock, LW_SHARED);
+
+   for (uint16 cur = queue->tail; cur != queue->next; cur++)
+   {
+       XLogIO *curio = XLogIOQueueElem(queue, cur);
+       PgAioIoRef aio_ref;
+
+       Assert(curio->upto != 0);
+
+       aio_ref = curio->aio_ref;
+       pg_read_barrier();
+
+       if (curio->in_progress)
+       {
+           aio_refs[waitcount++] = aio_ref;
+       }
+
+       if (lsn < curio->upto)
+           break;
+   }
+
+   LWLockRelease(WALIOQueueLock);
+
+   //elog(DEBUG1, "waiting for %d", waitcount);
+
+   /*
+    * We do that by waiting for the oldest IO to finish after releasing the
+    * write lock. The reason for doing so is that that is the completion that
+    * needs to be seen first "system wide". And later IOs might actually be
+    * issued in a different AIO context, therefore we wouldn't be guaranteed
+    * to see its completion if we e.g. waited for the last IO.
+    */
+   for (int i = waitcount - 1; i >= 0; i--)
+   //for (int i = 0; i < waitcount; i++)
+   {
+       if (!released_lock && release_write_lock)
+       {
+           if (pgaio_io_check_ref(&aio_refs[i]))
+               continue;
+
+           //elog(DEBUG1, "WALWriteLock wait");
+           LWLockRelease(WALWriteLock);
+           released_lock = true;
+       }
+       pgaio_io_wait_ref(&aio_refs[i], false);
+   }
+
+   LWLockAcquire(WALIOQueueLock, LW_SHARED);
+   SpinLockAcquire(&XLogCtl->info_lck);
+   LogwrtResult = XLogCtl->LogwrtResult;
+   SpinLockRelease(&XLogCtl->info_lck);
+   LWLockRelease(WALIOQueueLock);
+
+   return released_lock;
 }
 
+typedef struct XLogWritePos
+{
+   XLogRecPtr  write_init_min;
+   XLogRecPtr  write_init_opt;
+   XLogRecPtr  write_done_min;
+   XLogRecPtr  write_done_opt;
+   XLogRecPtr  flush_init_min;
+   XLogRecPtr  flush_init_opt;
+   XLogRecPtr  flush_done_min;
+   XLogRecPtr  flush_done_opt;
+} XLogWritePos;
+
 /*
- * Write and/or fsync the log at least as far as WriteRqst indicates.
- *
- * If flexible == true, we don't have to write as far as WriteRqst, but
- * may stop at any convenient boundary (such as a cache or logfile boundary).
- * This option allows us to avoid uselessly issuing multiple writes when a
- * single one would do.
- *
- * Must be called with WALWriteLock held. WaitXLogInsertionsToFinish(WriteRqst)
- * must be called before grabbing the lock, to make sure the data is ready to
- * write.
+ * Expects WALWriteLock to be held. If IO needs to be waited for first,
+ * WALWriteLock is released and true is returned after waiting. false is
+ * returned otherwise.
  */
+static bool
+XLogIOQueueEnsureOne(XLogIOQueue *queue)
+{
+   Assert(LWLockHeldByMeInMode(WALWriteLock, LW_EXCLUSIVE));
+
+   LWLockAcquire(WALIOQueueLock, LW_SHARED);
+
+   if (XLogIOQueueFull(queue))
+   {
+       XLogIO *io = XLogIOQueueTail(queue);
+       PgAioIoRef aio_ref = io->aio_ref;
+
+       elog(DEBUG3, "queue full, checking whether to wait: %d",
+            queue->tail & queue->mask);
+
+       if (io->in_progress)
+       {
+           LWLockRelease(WALIOQueueLock);
+           //elog(DEBUG1, "WALWriteLock ensure");
+           LWLockRelease(WALWriteLock);
+
+           pgaio_submit_pending(true);
+
+           elog(DEBUG3, "waiting for full queue to empty");
+           pgaio_io_wait_ref(&aio_ref, false);
+
+           return true;
+       }
+   }
+
+   LWLockRelease(WALIOQueueLock);
+
+   return false;
+}
+
 static void
-XLogWrite(XLogwrtRqst WriteRqst, bool flexible)
+XLogIOQueueAdd(XLogIOQueue *queue, PgAioInProgress *aio, XLogRecPtr upto)
+{
+   XLogIO *io;
+
+   elog(DEBUG3, "addinging aio %p at queue position %u/%u",
+        aio, queue->next, queue->next & queue->mask);
+
+   Assert(LWLockHeldByMeInMode(WALWriteLock, LW_EXCLUSIVE));
+   Assert(LWLockHeldByMeInMode(WALIOQueueLock, LW_EXCLUSIVE));
+
+   /* gotta have space, use XLogIOQueueEnsureOne */
+   Assert(!XLogIOQueueFull(queue));
+
+   io = XLogIOQueueElem(queue, queue->next);
+
+   Assert(!io->in_progress);
+   Assert(!pgaio_io_ref_valid(&io->aio_ref));
+   Assert(io->upto == 0);
+
+   io->upto = upto;
+   io->in_progress = true;
+   pgaio_io_ref(aio, &io->aio_ref);
+   queue->next++;
+
+   XLogIOQueueCheck(XLogCtl->writes);
+}
+
+static bool
+XLogWriteCheckPending(XLogWritePos *write_pos)
+{
+   bool    wait_needed = false;
+
+   if (LogwrtResult.WriteInit != LogwrtResult.WriteDone &&
+       (LogwrtResult.WriteInit % XLOG_BLCKSZ) != 0)
+   {
+       XLogIOQueue *writes = XLogCtl->writes;
+       XLogRecPtr startwrite = LogwrtResult.WriteInit - LogwrtResult.WriteInit % XLOG_BLCKSZ;
+       bool found_partial = false;
+       PgAioIoRef partial_aio_ref;
+       XLogRecPtr partial_upto;
+
+       /*
+        * If the previous write was a partial page write, we need to wait for
+        * it to finish. Otherwise the writes may get reordered (in the OS, or
+        * in the device). Which obviously will corrupt the WAL.
+        *
+        * FIXME: replace with no_reorder=true when issuing IO?
+        */
+       LWLockAcquire(WALIOQueueLock, LW_SHARED);
+
+       if (!XLogIOQueueEmpty(writes))
+       {
+           XLogIO *io = XLogIOQueueHead(writes);
+
+           partial_aio_ref = io->aio_ref;
+
+           pg_read_barrier();
+
+           partial_upto = io->upto;
+           if (partial_upto > startwrite && io->in_progress)
+           {
+               found_partial = true;
+           }
+       }
+
+       LWLockRelease(WALIOQueueLock);
+
+       if (found_partial)
+       {
+           wait_needed = true;
+
+           if (1)
+           {
+               pgaio_submit_pending(false);
+
+               ereport(DEBUG3,
+                       errmsg("waiting for partial write to %X/%X, so I can write from %X/%X to at least %X/%X",
+                              (uint32)(partial_upto >> 32), (uint32)partial_upto,
+                              (uint32)(startwrite >> 32), (uint32)startwrite,
+                              (uint32)(write_pos->write_init_min >> 32), (uint32) write_pos->write_init_min),
+                       errhidestmt(true),
+                       errhidecontext(true));
+               pgaio_io_wait_ref(&partial_aio_ref, false);
+           }
+           else
+           {
+#if 0
+               LWLockRelease(WALWriteLock);
+
+               pgaio_submit_pending(true);
+
+               ereport(DEBUG1,
+                       errmsg("waiting for partial write: %X/%X, so I can write from %X/%X to %X/%X",
+                              (uint32)(partial_upto >> 32), (uint32)partial_upto,
+                              (uint32)(startwrite >> 32), (uint32)startwrite,
+                              (uint32)(write_pos->write_init_min >> 32), (uint32) write_pos->write_init_min),
+                       errhidestmt(true),
+                       errhidecontext(true));
+
+               pgaio_io_wait_ref(&partial_aio_ref, false);
+
+               goto write_out_wait;
+#endif
+           }
+       }
+   }
+   else
+   {
+       ereport(DEBUG3,
+               errmsg("didn't need to check for partial write: init %X/%X, done %X/%X, desired %X/%X",
+                      (uint32)(LogwrtResult.WriteInit >> 32), (uint32)LogwrtResult.WriteInit,
+                      (uint32)(LogwrtResult.WriteDone >> 32), (uint32)LogwrtResult.WriteDone,
+                      (uint32)(write_pos->write_init_min >> 32), (uint32) write_pos->write_init_min),
+               errhidestmt(true),
+               errhidecontext(true));
+   }
+
+   return wait_needed;
+}
+
+static bool
+XLogWriteIssueWrites(XLogWritePos *write_pos, bool flexible)
 {
    bool        ispartialpage;
    bool        last_iteration;
@@ -2485,16 +2960,14 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible)
    int         npages;
    int         startidx;
    uint32      startoffset;
+   int         lastnonpartialidx;
+   XLogRecPtr  startwrite;
+   XLogRecPtr  startwrite_first = LogwrtResult.WriteInit;
+   int         writecount = 0;
 
-   /* We should always be inside a critical section here */
-   Assert(CritSectionCount > 0);
+   Assert(write_pos->write_init_opt >= write_pos->write_init_min);
 
-   Assert(XLogInsertionsKnownFinished(WriteRqst.Write));
-
-   /*
-    * Update local LogwrtResult (caller probably did this already, but...)
-    */
-   LogwrtResult = XLogCtl->LogwrtResult;
+   XLogWriteCheckPending(write_pos);
 
    /*
     * Since successive pages in the xlog cache are consecutively allocated,
@@ -2514,215 +2987,858 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible)
     * consider writing.  Begin at the buffer containing the next unwritten
     * page, or last partially written page.
     */
-   curridx = XLogRecPtrToBufIdx(LogwrtResult.Write);
+   curridx = XLogRecPtrToBufIdx(LogwrtResult.WriteInit);
+   if ((write_pos->write_init_opt % XLOG_BLCKSZ) != 0)
+       lastnonpartialidx = XLogRecPtrToBufIdx(write_pos->write_init_opt - XLOG_BLCKSZ);
+   else
+       lastnonpartialidx = -1;
 
-   while (LogwrtResult.Write < WriteRqst.Write)
+   while (LogwrtResult.WriteInit < write_pos->write_init_min)
    {
        /*
         * Make sure we're not ahead of the insert process.  This could happen
-        * if we're passed a bogus WriteRqst.Write that is past the end of the
+        * if we're passed a bogus WriteRqst->Write that is past the end of the
         * last page that's been initialized by AdvanceXLInsertBuffer.
         */
-       XLogRecPtr  EndPtr = XLogCtl->xlblocks[curridx];
+       XLogRecPtr  PageEndPtr = XLogCtl->xlblocks[curridx].lsn;
+       XLogRecPtr write_upto;
+       bool large_enough_write;
 
-       if (LogwrtResult.Write >= EndPtr)
+       if (LogwrtResult.WriteInit >= PageEndPtr)
            elog(PANIC, "xlog write request %X/%X is past end of log %X/%X",
-                (uint32) (LogwrtResult.Write >> 32),
-                (uint32) LogwrtResult.Write,
-                (uint32) (EndPtr >> 32), (uint32) EndPtr);
+                (uint32) (LogwrtResult.WriteInit >> 32),
+                (uint32) LogwrtResult.WriteInit,
+                (uint32) (PageEndPtr >> 32), (uint32) PageEndPtr);
 
-       /* Advance LogwrtResult.Write to end of current buffer page */
-       LogwrtResult.Write = EndPtr;
-       ispartialpage = WriteRqst.Write < LogwrtResult.Write;
+       ispartialpage = write_pos->write_init_opt < PageEndPtr;
 
-       if (!XLByteInPrevSeg(LogwrtResult.Write, openLogSegNo,
-                            wal_segment_size))
+       if (ispartialpage)
        {
+           XLogRecPtr newinsertpos = (XLogRecPtr)
+               pg_atomic_read_u64(&XLogCtl->Insert.knownCompletedUpto);
+           XLogRecPtr insert_lsn =
+               XLogBytePosToRecPtr(XLogCtl->Insert.CurrBytePos);
+
+           Assert(write_pos->write_init_opt <= newinsertpos);
+           Assert(write_pos->write_init_min > (PageEndPtr - XLOG_BLCKSZ));
+
+           if (write_pos->write_init_opt < newinsertpos)
+           {
+               write_pos->write_init_opt = newinsertpos;
+               ispartialpage = write_pos->write_init_opt < PageEndPtr;
+               //elog(LOG, "last ditch 1, new partial %d", ispartialpage);
+           }
+
            /*
-            * Switch to new logfile segment.  We cannot have any pending
-            * pages here (since we dump what we have at segment end).
+            * Safe to use WaitXLogInsertionsToFinish even while holding lock,
+            * as we are only waiting for something fitting onto the current
+            * page, which obviously has space. Therefore no deadlock danger.
             */
-           Assert(npages == 0);
-           if (openLogFile >= 0)
-               XLogFileClose();
-           XLByteToPrevSeg(LogwrtResult.Write, openLogSegNo,
-                           wal_segment_size);
 
-           /* create/use new log file */
-           use_existent = true;
-           openLogFile = XLogFileInit(openLogSegNo, &use_existent, true);
-           ReserveExternalFD();
-       }
+           if (ispartialpage && insert_lsn != write_pos->write_init_opt)
+           {
+               newinsertpos = WaitXLogInsertionsToFinish(newinsertpos);
+               write_pos->write_init_opt = newinsertpos;
+               ispartialpage = write_pos->write_init_opt < PageEndPtr;
+               insert_lsn = XLogBytePosToRecPtr(XLogCtl->Insert.CurrBytePos);
+           }
 
-       /* Make sure we have the current logfile open */
+           if (ispartialpage && insert_lsn > PageEndPtr)
+           {
+#if 0
+               LWLockRelease(WALWriteLock);
+               pgaio_submit_pending(true);
+               elog(DEBUG1, "WALWriteLock currbytepos");
+
+               newinsertpos = WaitXLogInsertionsToFinish(PageEndPtr);
+               write_pos->write_init_opt = newinsertpos;
+               ispartialpage = write_pos->write_init_opt < PageEndPtr;
+
+               goto write_out_wait;
+#else
+               ereport(DEBUG3,
+                       errmsg("locked waiting for insertion into partial page to finish. Cur %X/%X to %X/%X, ins is %X/%X",
+                              (uint32)(write_pos->write_init_min >> 32), (uint32) write_pos->write_init_min,
+                              (uint32)(PageEndPtr >> 32), (uint32) PageEndPtr,
+                              (uint32)(insert_lsn >> 32), (uint32) insert_lsn),
+                       errhidestmt(true),
+                       errhidecontext(true));
+
+               newinsertpos = WaitXLogInsertionsToFinish(PageEndPtr);
+               write_pos->write_init_opt = newinsertpos;
+               ispartialpage = write_pos->write_init_opt < PageEndPtr;
+#endif
+           }
+
+           if ((write_pos->write_init_opt % XLOG_BLCKSZ) != 0)
+               lastnonpartialidx = XLogRecPtrToBufIdx(write_pos->write_init_opt - XLOG_BLCKSZ);
+           else
+               lastnonpartialidx = -1;
+       }
+
+       if (ispartialpage)
+           write_upto = write_pos->write_init_opt;
+       else
+           write_upto = PageEndPtr;
+
+       if (!XLByteInPrevSeg(PageEndPtr, openLogSegNo, wal_segment_size))
+       {
+           /*
+            * Switch to new logfile segment.  We cannot have any pending
+            * pages here (since we dump what we have at segment end).
+            */
+           Assert(npages == 0);
+           if (openLogFile >= 0)
+               XLogFileClose();
+           XLByteToPrevSeg(PageEndPtr, openLogSegNo, wal_segment_size);
+
+           /* create/use new log file */
+           use_existent = true;
+           openLogFile = XLogFileInit(openLogSegNo, &use_existent, true);
+           ReserveExternalFD();
+       }
+
+       /* Make sure we have the current logfile open */
        if (openLogFile < 0)
        {
-           XLByteToPrevSeg(LogwrtResult.Write, openLogSegNo,
+           XLByteToPrevSeg(PageEndPtr, openLogSegNo, wal_segment_size);
+           openLogFile = XLogFileOpen(openLogSegNo);
+           ReserveExternalFD();
+       }
+
+       /* Add current page to the set of pending pages-to-dump */
+       if (npages == 0)
+       {
+           /* first of group */
+           startidx = curridx;
+           startwrite = PageEndPtr - XLOG_BLCKSZ;
+           startoffset = XLogSegmentOffset(startwrite, wal_segment_size);
+
+           /*
+            * Make sure there's space in the write queue.
+            */
+           if (XLogIOQueueEnsureOne(XLogCtl->writes))
+           {
+               goto write_out_wait;
+           }
+
+           /*
+            * And also in the fsync queue, as we do not want to wait
+            * additionally when doing an end-of-segment sync.
+            */
+           if (XLogIOQueueEnsureOne(XLogCtl->flushes))
+           {
+               goto write_out_wait;
+           }
+       }
+       npages++;
+
+       /*
+        * Dump the set if this will be the last loop iteration, or if we are
+        * at the last page of the cache area (since the next page won't be
+        * contiguous in memory), or if we are at the end of the logfile
+        * segment.
+        */
+       // FIXME, consider doing a separate write?
+
+       if (write_pos->write_init_opt <= PageEndPtr)
+           last_iteration = true;
+       else if (curridx == lastnonpartialidx && write_pos->write_init_min <= PageEndPtr)
+       {
+           last_iteration = true;
+           elog(DEBUG3, "preventing partial write min %X/%X, max %X/%X, page %X/%X",
+                (uint32) (write_pos->write_init_min >> 32), (uint32) write_pos->write_init_min,
+                (uint32) (write_pos->write_init_opt >> 32), (uint32) write_pos->write_init_opt,
+                (uint32) (PageEndPtr >> 32), (uint32) PageEndPtr);
+       }
+       else
+           last_iteration = false;
+
+       finishing_seg = !ispartialpage &&
+           (startoffset + npages * XLOG_BLCKSZ) >= wal_segment_size;
+
+       large_enough_write = npages >= io_wal_target_blocks;
+
+       if (!last_iteration && curridx == lastnonpartialidx)
+       {
+           elog(DEBUG3, "preventing overwrite %X/%X, page %X/%X",
+                (uint32) (write_pos->write_init_min >> 32), (uint32) write_pos->write_init_min,
+                (uint32) (PageEndPtr >> 32), (uint32) PageEndPtr);
+       }
+
+       if (last_iteration ||
+           curridx == XLogCtl->XLogCacheBlck ||
+           //(!io_wal_pad_partial && curridx == lastnonpartialidx) ||
+           curridx == lastnonpartialidx ||
+           finishing_seg ||
+           large_enough_write)
+       {
+           char       *from;
+           uint32      nbytes;
+
+           /* OK to write the page(s) */
+           from = XLogCtl->pages + startidx * (Size) XLOG_BLCKSZ;
+           nbytes = npages * (Size) XLOG_BLCKSZ;
+
+           Assert(write_upto > LogwrtResult.WriteInit);
+
+           if (0)
+           {
+               XLogRecPtr compl = pg_atomic_read_u64(&XLogCtl->Insert.knownCompletedUpto);
+               XLogRecPtr ins = XLogBytePosToRecPtr(XLogCtl->Insert.CurrBytePos);
+
+               ereport(DEBUG1,
+                       errmsg("performing %s write of %d bytes: from %X/%X to %X/%X, min %X/%X opt %X/%X (compl %X/%X: %ld, ins %X/%X: %ld)",
+                              ispartialpage ? "partial" : "non-partial",
+                              nbytes,
+                              (uint32) (startwrite >> 32), (uint32)startwrite,
+                              (uint32) (write_upto >> 32), (uint32)write_upto,
+                              (uint32) (write_pos->write_init_min >> 32), (uint32) write_pos->write_init_min,
+                              (uint32) (write_pos->write_init_opt >> 32), (uint32) write_pos->write_init_opt,
+                              (uint32) (compl >> 32), (uint32) compl, compl - write_upto,
+                              (uint32) (ins >> 32), (uint32) ins, ins - write_upto),
+                       errhidestmt(true),
+                       errhidecontext(true));
+
+           }
+
+           {
+               PgAioInProgress *aio;
+
+               aio = pgaio_io_get();
+               pgaio_io_start_write_wal(aio, openLogFile, startoffset,
+                                        nbytes, from, /* barrier = */ false,
+                                        XLogCtl->writes->next & XLogCtl->writes->mask);
+
+               LogwrtResult.WriteInit = write_upto;
+               if (sync_method == SYNC_METHOD_OPEN ||
+                   sync_method == SYNC_METHOD_OPEN_DSYNC)
+               {
+                   LogwrtResult.FlushInit = LogwrtResult.WriteInit;
+               }
+
+               LWLockAcquire(WALIOQueueLock, LW_EXCLUSIVE);
+
+               XLogIOQueueAdd(XLogCtl->writes, aio, write_upto);
+
+               SpinLockAcquire(&XLogCtl->info_lck);
+               XLogCtl->LogwrtResult.WriteInit = LogwrtResult.WriteInit;
+               XLogCtl->LogwrtResult.FlushInit = LogwrtResult.FlushInit;
+
+               if (XLogCtl->LogwrtRqst.WriteInit < LogwrtResult.WriteInit)
+                   XLogCtl->LogwrtRqst.WriteInit = LogwrtResult.WriteInit;
+               if (XLogCtl->LogwrtRqst.FlushInit < LogwrtResult.FlushInit)
+                   XLogCtl->LogwrtRqst.FlushInit = LogwrtResult.FlushInit;
+               SpinLockRelease(&XLogCtl->info_lck);
+
+               LWLockRelease(WALIOQueueLock);
+
+               from += nbytes;
+               startoffset += nbytes;
+
+               //pgaio_submit_pending(false);
+               pgaio_io_release(aio);
+           }
+
+
+           npages = 0;
+
+           /*
+            * If we just wrote the whole last page of a logfile segment,
+            * fsync the segment immediately.  This avoids having to go back
+            * and re-open prior segments when an fsync request comes along
+            * later. Doing it here ensures that one and only one backend will
+            * perform this fsync.
+            *
+            * This is also the right place to notify the Archiver that the
+            * segment is ready to copy to archival storage, and to update the
+            * timer for archive_timeout, and to signal for a checkpoint if
+            * too many logfile segments have been used since the last
+            * checkpoint.
+            */
+           if (finishing_seg)
+           {
+               pgaio_submit_pending(false);
+
+               if (LogwrtResult.FlushInit < LogwrtResult.WriteInit &&
+                   sync_method != SYNC_METHOD_OPEN &&
+                   sync_method != SYNC_METHOD_OPEN_DSYNC &&
+                   enableFsync)
+               {
+                   bool use_barrier;
+
+#if 1
+                   ereport(LOG,
+                           errmsg("waiting for end-of-segment writes"),
+                           errhidestmt(true),
+                           errhidecontext(true));
+
+                   /* wait holding lock, for now */
+                   XLogIOQueueWaitFor(XLogCtl->writes,
+                                      LogwrtResult.WriteInit,
+                                      false);
+                   Assert(LogwrtResult.WriteDone == LogwrtResult.WriteInit);
+                   use_barrier = false;
+#else
+                   use_barrier = true;
+#endif
+
+                   {
+                       PgAioInProgress *aio;
+                       bool fdatasync = false;
+
+#ifdef HAVE_FDATASYNC
+                       if (sync_method == SYNC_METHOD_FDATASYNC)
+                           fdatasync = true;
+#endif
+                       aio = pgaio_io_get();
+                       pgaio_io_start_fsync_wal(aio, openLogFile,
+                                                /* barrier = */ use_barrier,
+                                                fdatasync,
+                                                XLogCtl->flushes->next & XLogCtl->flushes->mask);
+
+                       LogwrtResult.FlushInit = LogwrtResult.WriteInit;
+
+                       LWLockAcquire(WALIOQueueLock, LW_EXCLUSIVE);
+
+                       XLogIOQueueAdd(XLogCtl->flushes, aio, LogwrtResult.WriteInit);
+
+                       SpinLockAcquire(&XLogCtl->info_lck);
+                       XLogCtl->LogwrtResult.FlushInit = LogwrtResult.FlushInit;
+                       SpinLockRelease(&XLogCtl->info_lck);
+
+                       LWLockRelease(WALIOQueueLock);
+
+                       pgaio_submit_pending(true);
+                       pgaio_io_release(aio);
+                   }
+
+               }
+
+               /* signal that we need to wakeup walsenders later */
+               WalSndWakeupRequest();
+
+               if (XLogArchivingActive())
+                   XLogArchiveNotifySeg(openLogSegNo);
+
+               XLogCtl->lastSegSwitchTime = (pg_time_t) time(NULL);
+               XLogCtl->lastSegSwitchLSN = LogwrtResult.FlushDone;
+
+               /*
+                * Request a checkpoint if we've consumed too much xlog since
+                * the last one.  For speed, we first check using the local
+                * copy of RedoRecPtr, which might be out of date; if it looks
+                * like a checkpoint is needed, forcibly update RedoRecPtr and
+                * recheck.
+                */
+               if (IsUnderPostmaster && XLogCheckpointNeeded(openLogSegNo))
+               {
+                   (void) GetRedoRecPtr();
+                   if (XLogCheckpointNeeded(openLogSegNo))
+                       RequestCheckpoint(CHECKPOINT_CAUSE_XLOG);
+               }
+           }
+       }
+
+       if (ispartialpage)
+       {
+           break;
+       }
+       curridx = NextBufIdx(curridx);
+
+#if 0
+       /* If flexible, break out of loop as soon as we wrote something */
+       if (flexible && npages == 0 &&
+           (curridx == XLogCtl->XLogCacheBlck ||
+            finishing_seg))
+       {
+           write_pos->write_init_min = Min(write_pos->write_init_min, LogwrtResult.WriteInit);
+           write_pos->write_done_min = Min(write_pos->write_done_min, write_pos->write_init_min);
+           write_pos->flush_init_min = Min(write_pos->flush_init_min, write_pos->write_done_min);
+           write_pos->flush_done_min = Min(write_pos->flush_done_min, write_pos->flush_init_min);
+           break;
+       }
+#endif
+       writecount++;
+       //elog(DEBUG1, "gonna write again: %d", writecount);
+   }
+
+   if (LogwrtResult.WriteInit < write_pos->write_done_opt)
+       elog(DEBUG1, "still would have more to write: opt is: %X/%X, %d bytes to %X/%X",
+            (uint32)(LogwrtResult.WriteInit >> 32), (uint32) LogwrtResult.WriteInit,
+            (int)(write_pos->write_done_opt - LogwrtResult.WriteInit),
+            (uint32)(write_pos->write_done_opt >> 32), (uint32) write_pos->write_done_opt);
+
+
+   Assert(npages == 0);
+   Assert(write_pos->write_done_min <= LogwrtResult.WriteInit);
+
+   /* don't drain while we hold lock */
+   pgaio_submit_pending(true);
+
+#if 0
+   if (AmWalWriterProcess())
+       elog(LOG, "issue writes exit false");
+#endif
+   return false;
+
+write_out_wait:
+#if 0
+   if (AmWalWriterProcess())
+       elog(LOG, "issue writes exit true");
+#endif
+   pgaio_submit_pending(true);
+   return true;
+}
+
+static bool
+XLogWriteIssueFlushes(XLogWritePos *write_pos)
+{
+   /* shouldn't even get here */
+   Assert(sync_method != SYNC_METHOD_OPEN &&
+          sync_method != SYNC_METHOD_OPEN_DSYNC);
+
+   if (!enableFsync)
+   {
+       LogwrtResult.FlushDone = LogwrtResult.FlushInit = LogwrtResult.WriteDone;
+
+       SpinLockAcquire(&XLogCtl->info_lck);
+       XLogCtl->LogwrtResult.FlushInit = LogwrtResult.FlushInit;
+       XLogCtl->LogwrtResult.FlushDone = LogwrtResult.FlushDone;
+       SpinLockRelease(&XLogCtl->info_lck);
+   }
+   else
+   {
+       /*
+        * Could get here without having written ourselves, in which case we might
+        * have no open file or the wrong one.  However, we do not need to fsync
+        * more than one file.
+        */
+       if (openLogFile >= 0 &&
+           !XLByteInPrevSeg(LogwrtResult.WriteDone, openLogSegNo,
+                            wal_segment_size))
+       {
+           XLogFileClose();
+       }
+
+       if (openLogFile < 0)
+       {
+           XLByteToPrevSeg(LogwrtResult.WriteDone, openLogSegNo,
                            wal_segment_size);
            openLogFile = XLogFileOpen(openLogSegNo);
            ReserveExternalFD();
        }
 
-       /* Add current page to the set of pending pages-to-dump */
-       if (npages == 0)
+       if (XLogIOQueueEnsureOne(XLogCtl->flushes))
+           return true;
+
+       if (1)
+       {
+           PgAioInProgress *aio;
+           bool use_fdatasync = false;
+
+#ifdef HAVE_FDATASYNC
+           if (sync_method == SYNC_METHOD_FDATASYNC)
+               use_fdatasync = true;
+#endif
+
+           aio = pgaio_io_get();
+           pgaio_io_start_fsync_wal(aio, openLogFile,
+                                    /* barrier = */ false,
+                                    use_fdatasync,
+                                    XLogCtl->flushes->next & XLogCtl->flushes->mask);
+
+           LogwrtResult.FlushInit = LogwrtResult.WriteDone;
+
+           LWLockAcquire(WALIOQueueLock, LW_EXCLUSIVE);
+
+           XLogIOQueueAdd(XLogCtl->flushes, aio, LogwrtResult.WriteDone);
+
+           LWLockRelease(WALIOQueueLock);
+
+           SpinLockAcquire(&XLogCtl->info_lck);
+           XLogCtl->LogwrtResult.WriteInit = LogwrtResult.WriteInit;
+           XLogCtl->LogwrtResult.FlushInit = LogwrtResult.FlushInit;
+           SpinLockRelease(&XLogCtl->info_lck);
+
+           pgaio_submit_pending(false);
+           pgaio_io_release(aio);
+       }
+       else
+       {
+           issue_xlog_fsync(openLogFile, openLogSegNo);
+           LogwrtResult.FlushDone = LogwrtResult.FlushInit = LogwrtResult.WriteDone;
+
+           SpinLockAcquire(&XLogCtl->info_lck);
+           XLogCtl->LogwrtResult.FlushInit = LogwrtResult.FlushInit;
+           XLogCtl->LogwrtResult.FlushDone = LogwrtResult.FlushDone;
+           SpinLockRelease(&XLogCtl->info_lck);
+
+       }
+   }
+
+
+   /* signal that we need to wakeup walsenders later */
+   // FIXME: also when using O_[D]SYNC?
+   WalSndWakeupRequest();
+
+   return false;
+}
+
+/*
+ * FIXME
+ */
+static bool pg_noinline
+XLogWrite(XLogwrtRqst WriteRqstTmp, bool flexible)
+{
+   bool        performed_io = false;
+   bool        holding_lock = false;
+   XLogWritePos write_pos = {0};
+   bool        did_wait_for_insert = false;
+
+   /* normalize request */
+   write_pos.write_init_min = WriteRqstTmp.WriteInit;
+   write_pos.write_done_min = WriteRqstTmp.WriteDone;
+   write_pos.flush_init_min = WriteRqstTmp.FlushInit;
+   write_pos.flush_done_min = WriteRqstTmp.FlushDone;
+
+   /* need to start flush before finishing it */
+   if (write_pos.flush_init_min < write_pos.flush_done_min)
+       write_pos.flush_init_min = write_pos.flush_done_min;
+
+   /* need to complete all writes before flushing them */
+   if (write_pos.write_done_min < write_pos.flush_init_min)
+       write_pos.write_done_min = write_pos.flush_init_min;
+
+   /* need to start write before finishing it */
+   if (write_pos.write_init_min < write_pos.write_done_min)
+       write_pos.write_init_min = write_pos.write_done_min;
+
+   write_pos.write_init_opt = write_pos.write_init_min;
+   write_pos.write_done_opt = write_pos.write_done_min;
+   write_pos.flush_init_opt = write_pos.flush_init_min;
+   write_pos.flush_done_opt = write_pos.flush_done_min;
+
+   START_CRIT_SECTION();
+
+xlogwrite_again:
+   Assert(!holding_lock);
+
+   SpinLockAcquire(&XLogCtl->info_lck);
+   LogwrtResult = XLogCtl->LogwrtResult;
+   SpinLockRelease(&XLogCtl->info_lck);
+
+   if (write_pos.write_init_min <= LogwrtResult.WriteInit &&
+       write_pos.write_done_min <= LogwrtResult.WriteDone &&
+       write_pos.flush_init_min <= LogwrtResult.FlushInit &&
+       write_pos.flush_done_min <= LogwrtResult.FlushDone)
+   {
+       END_CRIT_SECTION();
+       return performed_io;
+   }
+
+   if (write_pos.write_done_min <= LogwrtResult.WriteInit &&
+       write_pos.write_init_min <=  LogwrtResult.WriteInit &&
+       write_pos.flush_done_min <= LogwrtResult.FlushInit &&
+       write_pos.flush_init_min <= LogwrtResult.FlushInit)
+   {
+       if (write_pos.write_done_min <= LogwrtResult.WriteInit)
+       {
+           XLogIOQueueWaitFor(XLogCtl->writes, write_pos.write_done_min, false);
+           Assert(write_pos.write_done_min <= LogwrtResult.WriteDone);
+       }
+       if (write_pos.flush_done_min <= LogwrtResult.FlushInit)
+       {
+           XLogIOQueueWaitFor(XLogCtl->flushes, write_pos.flush_done_min, false);
+           Assert(write_pos.flush_done_min <= LogwrtResult.FlushDone);
+       }
+
+       END_CRIT_SECTION();
+
+       if (AmWalWriterProcess())
+       {
+           elog(DEBUG3, "walwriter didn't need to write, just wait: %X/%X vs %X/%X/",
+                (uint32) (write_pos.write_init_min >> 32), (uint32) write_pos.write_init_min,
+                (uint32) (LogwrtResult.WriteInit >> 32), (uint32) LogwrtResult.WriteInit);
+       }
+
+       //elog(DEBUG1, "just needed to wait for IO");
+       return performed_io;
+   }
+   else
+   {
+       bool retry = false;
+
+       /* wait at least as far as we need to write */
+       if (!did_wait_for_insert)
+       {
+           write_pos.write_init_opt = WaitXLogInsertionsToFinish(write_pos.write_init_min);
+           did_wait_for_insert = true;
+       }
+
+       /*
+        * Check if there's partial writes that need to finish without
+        * WALWriteLock.
+        */
+       if (XLogWriteCheckPending(&write_pos))
+           retry = true;
+
+       /*
+        * Check if the write would have to be partial, but could be
+        * non-partial if we waited for further insertions.
+        */
+       if (false && write_pos.write_init_min / XLOG_BLCKSZ == write_pos.write_init_opt / XLOG_BLCKSZ)
+       {
+           XLogRecPtr insert_lsn = XLogBytePosToRecPtr(XLogCtl->Insert.CurrBytePos);
+           XLogRecPtr page_end = write_pos.write_init_min
+               - write_pos.write_init_min % XLOG_BLCKSZ
+               + XLOG_BLCKSZ;
+
+           if (page_end < insert_lsn)
+           {
+               XLogRecPtr newinsertpos = WaitXLogInsertionsToFinish(page_end);
+
+               ereport(DEBUG3,
+                       errmsg("waited for insertion into partial page to finish. Cur %X/%X to %X/%X, ins is %X/%X, now %X/%X",
+                              (uint32)(write_pos.write_init_min), (uint32) write_pos.write_init_min,
+                              (uint32)(page_end >> 32), (uint32) page_end,
+                              (uint32)(insert_lsn >> 32), (uint32) insert_lsn,
+                              (uint32)(newinsertpos >> 32), (uint32) newinsertpos),
+                       errhidestmt(true),
+                       errhidecontext(true));
+               write_pos.write_init_opt = newinsertpos;
+
+               retry = true;
+           }
+       }
+
+       if (retry)
        {
-           /* first of group */
-           startidx = curridx;
-           startoffset = XLogSegmentOffset(LogwrtResult.Write - XLOG_BLCKSZ,
-                                           wal_segment_size);
+           goto xlogwrite_again;
        }
-       npages++;
+   }
 
-       /*
-        * Dump the set if this will be the last loop iteration, or if we are
-        * at the last page of the cache area (since the next page won't be
-        * contiguous in memory), or if we are at the end of the logfile
-        * segment.
-        */
-       last_iteration = WriteRqst.Write <= LogwrtResult.Write;
+   if (!LWLockAcquireOrWait(WALWriteLock, LW_EXCLUSIVE))
+   {
+       holding_lock = false;
+       goto xlogwrite_again;
+   }
+   else
+   {
+       holding_lock = true;
+   }
 
-       finishing_seg = !ispartialpage &&
-           (startoffset + npages * XLOG_BLCKSZ) >= wal_segment_size;
+   SpinLockAcquire(&XLogCtl->info_lck);
+   LogwrtResult = XLogCtl->LogwrtResult;
+   SpinLockRelease(&XLogCtl->info_lck);
 
-       if (last_iteration ||
-           curridx == XLogCtl->XLogCacheBlck ||
-           finishing_seg)
+   /* Quite possible write request has already been fulfilled. */
+   if (write_pos.write_init_min <= LogwrtResult.WriteInit &&
+       write_pos.write_done_min <= LogwrtResult.WriteDone &&
+       write_pos.flush_init_min <= LogwrtResult.FlushInit &&
+       write_pos.flush_done_min <= LogwrtResult.FlushDone)
+   {
+       Assert(holding_lock);
+       //elog(DEBUG1, "WALWriteLock already fulfilled");
+       LWLockRelease(WALWriteLock);
+       holding_lock = false;
+
+       if (AmWalWriterProcess())
        {
-           char       *from;
-           Size        nbytes;
-           Size        nleft;
-           int         written;
+           elog(LOG, "walwriter didn't need to write: %X/%X vs %X/%X/",
+                (uint32)(write_pos.write_init_min >> 32), (uint32) write_pos.write_init_min,
+                (uint32) (LogwrtResult.WriteInit >> 32), (uint32) LogwrtResult.WriteInit);
 
-           /* OK to write the page(s) */
-           from = XLogCtl->pages + startidx * (Size) XLOG_BLCKSZ;
-           nbytes = npages * (Size) XLOG_BLCKSZ;
-           nleft = nbytes;
-           do
-           {
-               errno = 0;
-               pgstat_report_wait_start(WAIT_EVENT_WAL_WRITE);
-               written = pg_pwrite(openLogFile, from, nleft, startoffset);
-               pgstat_report_wait_end();
-               if (written <= 0)
-               {
-                   char        xlogfname[MAXFNAMELEN];
-                   int         save_errno;
+       }
 
-                   if (errno == EINTR)
-                       continue;
+       END_CRIT_SECTION();
+       return performed_io;
+   }
 
-                   save_errno = errno;
-                   XLogFileName(xlogfname, ThisTimeLineID, openLogSegNo,
-                                wal_segment_size);
-                   errno = save_errno;
-                   ereport(PANIC,
-                           (errcode_for_file_access(),
-                            errmsg("could not write to log file %s "
-                                   "at offset %u, length %zu: %m",
-                                   xlogfname, startoffset, nleft)));
-               }
-               nleft -= written;
-               from += written;
-               startoffset += written;
-           } while (nleft > 0);
+   /*
+    * Write more than necessary. Reduces overall IO.
+    */
+   if (LogwrtResult.WriteInit < write_pos.write_init_min)
+   {
+       XLogRecPtr  insertpos,
+           insertpos_before;
 
-           npages = 0;
+       insertpos_before = (XLogRecPtr)
+           pg_atomic_read_u64(&XLogCtl->Insert.knownCompletedUpto);
+       insertpos = insertpos_before;
 
-           /*
-            * If we just wrote the whole last page of a logfile segment,
-            * fsync the segment immediately.  This avoids having to go back
-            * and re-open prior segments when an fsync request comes along
-            * later. Doing it here ensures that one and only one backend will
-            * perform this fsync.
-            *
-            * This is also the right place to notify the Archiver that the
-            * segment is ready to copy to archival storage, and to update the
-            * timer for archive_timeout, and to signal for a checkpoint if
-            * too many logfile segments have been used since the last
-            * checkpoint.
-            */
-           if (finishing_seg)
-           {
-               issue_xlog_fsync(openLogFile, openLogSegNo);
+       Assert(write_pos.write_init_min <= insertpos_before);
 
-               /* signal that we need to wakeup walsenders later */
-               WalSndWakeupRequest();
+#if 0
+       SpinLockAcquire(&XLogCtl->info_lck);
+       if (insertpos_before < XLogCtl->LogwrtRqst.WriteInit)
+           insertpos_before = XLogCtl->LogwrtRqst.WriteInit;
+       SpinLockRelease(&XLogCtl->info_lck);
 
-               LogwrtResult.Flush = LogwrtResult.Write;    /* end of page */
+       insertpos = WaitXLogInsertionsToFinish(insertpos_before);
+#endif
 
-               if (XLogArchivingActive())
-                   XLogArchiveNotifySeg(openLogSegNo);
+       Assert(write_pos.write_init_min <= insertpos);
 
-               XLogCtl->lastSegSwitchTime = (pg_time_t) time(NULL);
-               XLogCtl->lastSegSwitchLSN = LogwrtResult.Flush;
+       /*
+        * FIXME: Limit how much more than this backend's WAL to write.
+        */
+       if (insertpos != write_pos.write_init_min)
+       {
+           ereport(DEBUG3,
+                   errmsg("advancing from WriteInit %X/%X to %X/%X, before %X/%X",
+                          (uint32)(write_pos.write_init_min >> 32), (uint32) write_pos.write_init_min,
+                          (uint32)(insertpos >> 32), (uint32) insertpos,
+                          (uint32)(insertpos_before >> 32), (uint32) insertpos_before),
+                   errhidestmt(true),
+                   errhidecontext(true));
+
+           if ((insertpos - insertpos % XLOG_BLCKSZ) > write_pos.write_init_opt)
+               write_pos.write_init_opt = insertpos;
+       }
+   }
 
-               /*
-                * Request a checkpoint if we've consumed too much xlog since
-                * the last one.  For speed, we first check using the local
-                * copy of RedoRecPtr, which might be out of date; if it looks
-                * like a checkpoint is needed, forcibly update RedoRecPtr and
-                * recheck.
-                */
-               if (IsUnderPostmaster && XLogCheckpointNeeded(openLogSegNo))
-               {
-                   (void) GetRedoRecPtr();
-                   if (XLogCheckpointNeeded(openLogSegNo))
-                       RequestCheckpoint(CHECKPOINT_CAUSE_XLOG);
-               }
-           }
+   Assert(LWLockHeldByMeInMode(WALWriteLock, LW_EXCLUSIVE));
+
+   /*
+    * If we need to initiate writes, do so. But if there are too many IOs
+    * already in progress, we need to wait for some IO to finish. In that
+    * case we retry, because it's somewhat likely that somebody else was able
+    * to initiate IO before we got woken up.
+    */
+   if (LogwrtResult.WriteInit < write_pos.write_init_min)
+   {
+#if 0
+       {
+           XLogRecPtr  reserved_upto = XLogBytePosToEndRecPtr(XLogCtl->Insert.CurrBytePos);
+           XLogRecPtr  completed = WaitXLogInsertionsToFinish(WriteRqst.WriteInit);
+
+           ereport(DEBUG1,
+                   errmsg("xlog write write cur/target: "
+                          "WriteInit: %X/%X->%X/%X: %u, "
+                          "WriteDone: %X/%X->%X/%X, "
+                          "FlushInit: %X/%X->%X/%X, "
+                          "FlushDone: %X/%X->%X/%X, "
+                          "Reserved: %X/%X, "
+                          "Completed: %X/%X, "
+                          "Reserved - Completed: %lu, "
+                          "Completed - prev WriteInit: %lu, "
+                          "Completed - new WriteInit: %lu, ",
+                          (uint32) (LogwrtResult.WriteInit >> 32), (uint32) (LogwrtResult.WriteInit),
+                          (uint32) (WriteRqst.WriteInit >> 32), (uint32) (WriteRqst.WriteInit),
+                          (uint32) (WriteRqst.WriteInit - LogwrtResult.WriteInit),
+
+                          (uint32) (LogwrtResult.WriteDone >> 32), (uint32) (LogwrtResult.WriteDone),
+                          (uint32) (WriteRqst.WriteDone >> 32), (uint32) (WriteRqst.WriteDone),
+
+                          (uint32) (LogwrtResult.FlushInit >> 32), (uint32) (LogwrtResult.FlushInit),
+                          (uint32) (WriteRqst.FlushInit >> 32), (uint32) (WriteRqst.FlushInit),
+
+                          (uint32) (LogwrtResult.FlushDone >> 32), (uint32) (LogwrtResult.FlushDone),
+                          (uint32) (WriteRqst.FlushDone >> 32), (uint32) (WriteRqst.FlushDone),
+
+                          (uint32) (reserved_upto >> 32), (uint32) reserved_upto,
+                          (uint32) (completed >> 32), (uint32) completed,
+                          reserved_upto - completed,
+                          completed - LogwrtResult.WriteInit,
+                          completed - WriteRqst.WriteInit),
+                   errhidestmt(true),
+                   errhidecontext(true));
        }
+#endif
 
-       if (ispartialpage)
+       if (XLogWriteIssueWrites(&write_pos, flexible))
        {
-           /* Only asked to write a partial page */
-           LogwrtResult.Write = WriteRqst.Write;
-           break;
+           //elog(LOG, "issued writes false: ");
+           holding_lock = false;
+           goto xlogwrite_again;
        }
-       curridx = NextBufIdx(curridx);
+#if 0
+       ereport(LOG,
+               errmsg("write done"),
+               errhidestmt(true),
+               errhidecontext(true));
+#endif
 
-       /* If flexible, break out of loop as soon as we wrote something */
-       if (flexible && npages == 0)
-           break;
+       performed_io = true;
+
+#if 0
+       if (flexible)
+       {
+           if (WriteRqst.write_init_min >= LogwrtResult.WriteInit)
+               WriteRqst.write_init_min = LogwrtResult.WriteInit;
+           if (WriteRqst.write_done_min >= LogwrtResult.WriteInit)
+               WriteRqst.write_done_min = LogwrtResult.WriteInit;
+           if (WriteRqst.write_flush_init_min >= LogwrtResult.WriteInit)
+               WriteRqst.write_flush_init_min = LogwrtResult.WriteInit;
+           if (WriteRqst.write_flush_done_min >= WriteRqst.FlushInit)
+               WriteRqst.write_flush_done_min = WriteRqst.FlushInit;
+       }
+#endif
    }
 
-   Assert(npages == 0);
+   Assert(write_pos.write_init_min <= LogwrtResult.WriteInit);
 
    /*
-    * If asked to flush, do so
+    * If we need to wait for writes to complete, do so.
     */
-   if (LogwrtResult.Flush < WriteRqst.Flush &&
-       LogwrtResult.Flush < LogwrtResult.Write)
-
+   if (LogwrtResult.WriteDone < write_pos.write_done_min)
    {
-       /*
-        * Could get here without iterating above loop, in which case we might
-        * have no open file or the wrong one.  However, we do not need to
-        * fsync more than one file.
-        */
-       if (sync_method != SYNC_METHOD_OPEN &&
-           sync_method != SYNC_METHOD_OPEN_DSYNC)
+       if (XLogIOQueueWaitFor(XLogCtl->writes, write_pos.write_done_min, true))
        {
-           if (openLogFile >= 0 &&
-               !XLByteInPrevSeg(LogwrtResult.Write, openLogSegNo,
-                                wal_segment_size))
-               XLogFileClose();
-           if (openLogFile < 0)
-           {
-               XLByteToPrevSeg(LogwrtResult.Write, openLogSegNo,
-                               wal_segment_size);
-               openLogFile = XLogFileOpen(openLogSegNo);
-               ReserveExternalFD();
-           }
+           holding_lock = false;
+           goto xlogwrite_again;
+       }
+   }
 
-           issue_xlog_fsync(openLogFile, openLogSegNo);
+   Assert(write_pos.write_done_min <= LogwrtResult.WriteDone);
+
+   if (flexible && write_pos.flush_init_min > LogwrtResult.WriteDone)
+   {
+       write_pos.flush_init_min = LogwrtResult.WriteDone;
+   }
+
+   /*
+    * If we need to issue flushes, do so. Pretty much the same as the write
+    * case above.
+    *
+    * Note that when flexible is true, we might not be able to initiate any
+    * flushes, because not enough was written.
+    */
+   if (LogwrtResult.FlushInit < write_pos.flush_init_min)
+   {
+       Assert(LogwrtResult.WriteDone > LogwrtResult.FlushInit);
+
+       write_pos.flush_init_opt = LogwrtResult.WriteDone;
+
+       if (XLogWriteIssueFlushes(&write_pos))
+       {
+           holding_lock = false;
+           goto xlogwrite_again;
        }
 
-       /* signal that we need to wakeup walsenders later */
-       WalSndWakeupRequest();
+       performed_io = true;
+
+       Assert(flexible ||
+              write_pos.flush_init_min <= LogwrtResult.FlushInit);
+   }
 
-       LogwrtResult.Flush = LogwrtResult.Write;
+   /*
+    * Wait for all required flushes to complete.
+    */
+   if (!flexible &&
+       (LogwrtResult.FlushDone < write_pos.flush_done_min))
+   {
+       if (XLogIOQueueWaitFor(XLogCtl->flushes, write_pos.flush_done_min, true))
+       {
+           holding_lock = false;
+           goto xlogwrite_again;
+       }
    }
 
+   Assert(flexible ||
+          write_pos.flush_done_min <= LogwrtResult.FlushDone);
+
    /*
     * Update shared-memory status
     *
@@ -2732,13 +3848,22 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible)
     */
    {
        SpinLockAcquire(&XLogCtl->info_lck);
-       XLogCtl->LogwrtResult = LogwrtResult;
-       if (XLogCtl->LogwrtRqst.Write < LogwrtResult.Write)
-           XLogCtl->LogwrtRqst.Write = LogwrtResult.Write;
-       if (XLogCtl->LogwrtRqst.Flush < LogwrtResult.Flush)
-           XLogCtl->LogwrtRqst.Flush = LogwrtResult.Flush;
+       if (XLogCtl->LogwrtRqst.WriteInit < LogwrtResult.WriteInit)
+           XLogCtl->LogwrtRqst.WriteInit = LogwrtResult.WriteInit;
+       if (XLogCtl->LogwrtRqst.WriteDone < LogwrtResult.WriteDone)
+           XLogCtl->LogwrtRqst.WriteDone = LogwrtResult.WriteDone;
+       if (XLogCtl->LogwrtRqst.FlushInit < LogwrtResult.FlushInit)
+           XLogCtl->LogwrtRqst.FlushInit = LogwrtResult.FlushInit;
+       if (XLogCtl->LogwrtRqst.FlushDone < LogwrtResult.FlushDone)
+           XLogCtl->LogwrtRqst.FlushDone = LogwrtResult.FlushDone;
        SpinLockRelease(&XLogCtl->info_lck);
    }
+
+   Assert(holding_lock);
+   LWLockRelease(WALWriteLock);
+
+   END_CRIT_SECTION();
+   return performed_io;
 }
 
 /*
@@ -2770,7 +3895,7 @@ XLogSetAsyncXactLSN(XLogRecPtr asyncXactLSN)
        WriteRqstPtr -= WriteRqstPtr % XLOG_BLCKSZ;
 
        /* if we have already flushed that far, we're done */
-       if (WriteRqstPtr <= LogwrtResult.Flush)
+       if (WriteRqstPtr <= LogwrtResult.FlushDone)
            return;
    }
 
@@ -2905,14 +4030,12 @@ UpdateMinRecoveryPoint(XLogRecPtr lsn, bool force)
 /*
  * Ensure that all XLOG data through the given position is flushed to disk.
  *
- * NOTE: this differs from XLogWrite mainly in that the WALWriteLock is not
- * already held, and we try to avoid acquiring it if possible.
+ * NOTE: FIXME
  */
 void
 XLogFlush(XLogRecPtr record)
 {
-   XLogRecPtr  WriteRqstPtr;
-   XLogwrtRqst WriteRqst;
+   XLogwrtRqst WriteRqst = {0};
 
    /*
     * During REDO, we are reading not writing WAL.  Therefore, instead of
@@ -2928,121 +4051,39 @@ XLogFlush(XLogRecPtr record)
    }
 
    /* Quick exit if already known flushed */
-   if (record <= LogwrtResult.Flush)
+   if (record <= LogwrtResult.FlushDone)
        return;
 
 #ifdef WAL_DEBUG
    if (XLOG_DEBUG)
        elog(LOG, "xlog flush request %X/%X; write %X/%X; flush %X/%X",
             (uint32) (record >> 32), (uint32) record,
-            (uint32) (LogwrtResult.Write >> 32), (uint32) LogwrtResult.Write,
-            (uint32) (LogwrtResult.Flush >> 32), (uint32) LogwrtResult.Flush);
+            (uint32) (LogwrtResult.WriteDone >> 32), (uint32) LogwrtResult.WriteDone,
+            (uint32) (LogwrtResult.FlushDone >> 32), (uint32) LogwrtResult.FlushDone);
 #endif
 
-   START_CRIT_SECTION();
-
    /*
     * Since fsync is usually a horribly expensive operation, we try to
     * piggyback as much data as we can on each fsync: if we see any more data
     * entered into the xlog buffer, we'll write and fsync that too, so that
     * the final value of LogwrtResult.Flush is as large as possible. This
     * gives us some chance of avoiding another fsync immediately after.
+    *
+    * XXX: Move.
     */
 
-   /* initialize to given target; may increase below */
-   WriteRqstPtr = record;
-
-   /*
-    * Now wait until we get the write lock, or someone else does the flush
-    * for us.
-    */
-   for (;;)
-   {
-       XLogRecPtr  insertpos;
-
-       /* read LogwrtResult and update local state */
-       SpinLockAcquire(&XLogCtl->info_lck);
-       if (WriteRqstPtr < XLogCtl->LogwrtRqst.Write)
-           WriteRqstPtr = XLogCtl->LogwrtRqst.Write;
-       LogwrtResult = XLogCtl->LogwrtResult;
-       SpinLockRelease(&XLogCtl->info_lck);
-
-       /* done already? */
-       if (record <= LogwrtResult.Flush)
-           break;
-
-       /*
-        * Before actually performing the write, wait for all in-flight
-        * insertions to the pages we're about to write to finish.
-        */
-       insertpos = WaitXLogInsertionsToFinish(WriteRqstPtr);
-
-       /*
-        * Try to get the write lock. If we can't get it immediately, wait
-        * until it's released, and recheck if we still need to do the flush
-        * or if the backend that held the lock did it for us already. This
-        * helps to maintain a good rate of group committing when the system
-        * is bottlenecked by the speed of fsyncing.
-        */
-       if (!LWLockAcquireOrWait(WALWriteLock, LW_EXCLUSIVE))
-       {
-           /*
-            * The lock is now free, but we didn't acquire it yet. Before we
-            * do, loop back to check if someone else flushed the record for
-            * us already.
-            */
-           continue;
-       }
-
-       /* Got the lock; recheck whether request is satisfied */
-       LogwrtResult = XLogCtl->LogwrtResult;
-       if (record <= LogwrtResult.Flush)
-       {
-           LWLockRelease(WALWriteLock);
-           break;
-       }
-
-       /*
-        * Sleep before flush! By adding a delay here, we may give further
-        * backends the opportunity to join the backlog of group commit
-        * followers; this can significantly improve transaction throughput,
-        * at the risk of increasing transaction latency.
-        *
-        * We do not sleep if enableFsync is not turned on, nor if there are
-        * fewer than CommitSiblings other backends with active transactions.
-        */
-       if (CommitDelay > 0 && enableFsync &&
-           MinimumActiveBackends(CommitSiblings))
-       {
-           pg_usleep(CommitDelay);
-
-           /*
-            * Re-check how far we can now flush the WAL. It's generally not
-            * safe to call WaitXLogInsertionsToFinish while holding
-            * WALWriteLock, because an in-progress insertion might need to
-            * also grab WALWriteLock to make progress. But we know that all
-            * the insertions up to insertpos have already finished, because
-            * that's what the earlier WaitXLogInsertionsToFinish() returned.
-            * We're only calling it again to allow insertpos to be moved
-            * further forward, not to actually wait for anyone.
-            */
-           insertpos = WaitXLogInsertionsToFinish(insertpos);
-       }
-
-       /* try to write/flush later additions to XLOG as well */
-       WriteRqst.Write = insertpos;
-       WriteRqst.Flush = insertpos;
-
-       XLogWrite(WriteRqst, false);
+   WriteRqst.WriteInit = 0;
+   WriteRqst.WriteDone = 0;
+   WriteRqst.FlushInit = 0;
+   WriteRqst.FlushDone = record;
 
-       LWLockRelease(WALWriteLock);
-       /* done */
-       break;
-   }
+   pgaio_submit_pending(true);
 
-   END_CRIT_SECTION();
+   // FIXME: reimplement commit delay
+   XLogWrite(WriteRqst, false);
 
    /* wake up walsenders now that we've released heavily contended locks */
+   // XXX: Move */
    WalSndWakeupProcessRequests();
 
    /*
@@ -3066,11 +4107,11 @@ XLogFlush(XLogRecPtr record)
     * calls from bufmgr.c are not within critical sections and so we will not
     * force a restart for a bad LSN on a data page.
     */
-   if (LogwrtResult.Flush < record)
+   if (LogwrtResult.FlushDone < record)
        elog(ERROR,
             "xlog flush request %X/%X is not satisfied --- flushed only to %X/%X",
             (uint32) (record >> 32), (uint32) record,
-            (uint32) (LogwrtResult.Flush >> 32), (uint32) LogwrtResult.Flush);
+            (uint32) (LogwrtResult.FlushDone >> 32), (uint32) LogwrtResult.FlushDone);
 }
 
 /*
@@ -3100,11 +4141,12 @@ XLogFlush(XLogRecPtr record)
 bool
 XLogBackgroundFlush(void)
 {
-   XLogwrtRqst WriteRqst;
+   XLogwrtRqst WriteRqst = {0};
    bool        flexible = true;
    static TimestampTz lastflush;
    TimestampTz now;
    int         flushbytes;
+   XLogRecPtr  asyncXactLSN = InvalidXLogRecPtr;
 
    /* XLOG doesn't need flushing during recovery */
    if (RecoveryInProgress())
@@ -3114,30 +4156,23 @@ XLogBackgroundFlush(void)
    SpinLockAcquire(&XLogCtl->info_lck);
    LogwrtResult = XLogCtl->LogwrtResult;
    WriteRqst = XLogCtl->LogwrtRqst;
+   asyncXactLSN = XLogCtl->asyncXactLSN;
    SpinLockRelease(&XLogCtl->info_lck);
 
    /* back off to last completed page boundary */
-   WriteRqst.Write -= WriteRqst.Write % XLOG_BLCKSZ;
-
-   /* if we have already flushed that far, consider async commit records */
-   if (WriteRqst.Write <= LogwrtResult.Flush)
-   {
-       SpinLockAcquire(&XLogCtl->info_lck);
-       WriteRqst.Write = XLogCtl->asyncXactLSN;
-       SpinLockRelease(&XLogCtl->info_lck);
-       flexible = false;       /* ensure it all gets written */
-   }
+   WriteRqst.WriteInit -= WriteRqst.WriteInit % XLOG_BLCKSZ;
+   WriteRqst.WriteDone -= WriteRqst.WriteDone % XLOG_BLCKSZ;
 
    /*
     * If already known flushed, we're done. Just need to check if we are
     * holding an open file handle to a logfile that's no longer in use,
     * preventing the file from being deleted.
     */
-   if (WriteRqst.Write <= LogwrtResult.Flush)
+   if (Max(WriteRqst.WriteInit, asyncXactLSN) <= LogwrtResult.FlushDone)
    {
        if (openLogFile >= 0)
        {
-           if (!XLByteInPrevSeg(LogwrtResult.Write, openLogSegNo,
+           if (!XLByteInPrevSeg(LogwrtResult.WriteInit, openLogSegNo,
                                 wal_segment_size))
            {
                XLogFileClose();
@@ -3152,59 +4187,54 @@ XLogBackgroundFlush(void)
     */
    now = GetCurrentTimestamp();
    flushbytes =
-       WriteRqst.Write / XLOG_BLCKSZ - LogwrtResult.Flush / XLOG_BLCKSZ;
+       WriteRqst.WriteInit / XLOG_BLCKSZ - LogwrtResult.FlushInit / XLOG_BLCKSZ;
 
    if (WalWriterFlushAfter == 0 || lastflush == 0)
    {
        /* first call, or block based limits disabled */
-       WriteRqst.Flush = WriteRqst.Write;
+       WriteRqst.FlushDone = WriteRqst.WriteInit;
        lastflush = now;
    }
    else if (TimestampDifferenceExceeds(lastflush, now, WalWriterDelay))
    {
+       /* if we have already flushed that far, consider async commit records */
+       if (WriteRqst.WriteDone <= LogwrtResult.FlushDone)
+       {
+           WriteRqst.WriteInit = asyncXactLSN;
+           flexible = false;       /* ensure it all gets written */
+       }
+
        /*
         * Flush the writes at least every WalWriterDelay ms. This is
         * important to bound the amount of time it takes for an asynchronous
         * commit to hit disk.
         */
-       WriteRqst.Flush = WriteRqst.Write;
+       WriteRqst.FlushDone = WriteRqst.WriteInit;
        lastflush = now;
    }
    else if (flushbytes >= WalWriterFlushAfter)
    {
        /* exceeded wal_writer_flush_after blocks, flush */
-       WriteRqst.Flush = WriteRqst.Write;
+       WriteRqst.FlushInit = WriteRqst.WriteInit;
        lastflush = now;
    }
    else
    {
        /* no flushing, this time round */
-       WriteRqst.Flush = 0;
+       WriteRqst.FlushInit = 0;
+       WriteRqst.FlushDone = 0;
    }
 
 #ifdef WAL_DEBUG
    if (XLOG_DEBUG)
        elog(LOG, "xlog bg flush request write %X/%X; flush: %X/%X, current is write %X/%X; flush %X/%X",
-            (uint32) (WriteRqst.Write >> 32), (uint32) WriteRqst.Write,
-            (uint32) (WriteRqst.Flush >> 32), (uint32) WriteRqst.Flush,
-            (uint32) (LogwrtResult.Write >> 32), (uint32) LogwrtResult.Write,
-            (uint32) (LogwrtResult.Flush >> 32), (uint32) LogwrtResult.Flush);
+            (uint32) (WriteRqst.WriteDone >> 32), (uint32) WriteRqst.WriteDone,
+            (uint32) (WriteRqst.FlushDone >> 32), (uint32) WriteRqst.FlushDone,
+            (uint32) (LogwrtResult.WriteDone >> 32), (uint32) LogwrtResult.WriteDone,
+            (uint32) (LogwrtResult.FlushDone >> 32), (uint32) LogwrtResult.FlushDone);
 #endif
 
-   START_CRIT_SECTION();
-
-   /* now wait for any in-progress insertions to finish and get write lock */
-   WaitXLogInsertionsToFinish(WriteRqst.Write);
-   LWLockAcquire(WALWriteLock, LW_EXCLUSIVE);
-   LogwrtResult = XLogCtl->LogwrtResult;
-   if (WriteRqst.Write > LogwrtResult.Write ||
-       WriteRqst.Flush > LogwrtResult.Flush)
-   {
-       XLogWrite(WriteRqst, flexible);
-   }
-   LWLockRelease(WALWriteLock);
-
-   END_CRIT_SECTION();
+   XLogWrite(WriteRqst, flexible);
 
    /* wake up walsenders now that we've released heavily contended locks */
    WalSndWakeupProcessRequests();
@@ -3280,7 +4310,7 @@ XLogNeedsFlush(XLogRecPtr record)
    }
 
    /* Quick exit if already known flushed */
-   if (record <= LogwrtResult.Flush)
+   if (record <= LogwrtResult.FlushDone)
        return false;
 
    /* read LogwrtResult and update local state */
@@ -3289,7 +4319,7 @@ XLogNeedsFlush(XLogRecPtr record)
    SpinLockRelease(&XLogCtl->info_lck);
 
    /* check again */
-   if (record <= LogwrtResult.Flush)
+   if (record <= LogwrtResult.FlushDone)
        return false;
 
    return true;
@@ -3325,11 +4355,11 @@ XLogFileInit(XLogSegNo logsegno, bool *use_existent, bool use_lock)
 {
    char        path[MAXPGPATH];
    char        tmppath[MAXPGPATH];
-   PGAlignedXLogBlock zbuffer;
    XLogSegNo   installed_segno;
    XLogSegNo   max_segno;
    int         fd;
    int         save_errno;
+   pg_streaming_write *pgsw;
    int         open_flags = O_RDWR | O_CREAT | O_EXCL | PG_BINARY;
 
    XLogFilePath(path, ThisTimeLineID, logsegno, wal_segment_size);
@@ -3359,6 +4389,14 @@ XLogFileInit(XLogSegNo logsegno, bool *use_existent, bool use_lock)
     */
    elog(DEBUG2, "creating and filling new WAL file");
 
+   /*
+    * FIXME: Probably want one permanently allocated? Or perhaps this should
+    * just be part of the AIO infrastructure somehow.
+    */
+   CurrentMemoryContext->allowInCritSection = true;
+   pgsw = pg_streaming_write_alloc(128, NULL, XLogFileInitComplete);
+   CurrentMemoryContext->allowInCritSection = true;
+
    snprintf(tmppath, MAXPGPATH, XLOGDIR "/xlogtemp.%d", (int) getpid());
 
    unlink(tmppath);
@@ -3373,15 +4411,11 @@ XLogFileInit(XLogSegNo logsegno, bool *use_existent, bool use_lock)
                (errcode_for_file_access(),
                 errmsg("could not create file \"%s\": %m", tmppath)));
 
-   memset(zbuffer.data, 0, XLOG_BLCKSZ);
 
    pgstat_report_wait_start(WAIT_EVENT_WAL_INIT_WRITE);
    save_errno = 0;
    if (wal_init_zero)
    {
-       struct iovec iov[PG_IOV_MAX];
-       int         blocks;
-
        /*
         * Zero-fill the file.  With this setting, we do this the hard way to
         * ensure that all the file space has really been allocated.  On
@@ -3392,28 +4426,52 @@ XLogFileInit(XLogSegNo logsegno, bool *use_existent, bool use_lock)
         * O_DSYNC will be sufficient to sync future writes to the log file.
         */
 
-       /* Prepare to write out a lot of copies of our zero buffer at once. */
-       for (int i = 0; i < lengthof(iov); ++i)
+       if (posix_fallocate(fd, 0, wal_segment_size) != 0)
        {
-           iov[i].iov_base = zbuffer.data;
-           iov[i].iov_len = XLOG_BLCKSZ;
+           /* if write didn't set errno, assume no disk space */
+           save_errno = errno ? errno : ENOSPC;
        }
-
-       /* Loop, writing as many blocks as we can for each system call. */
-       blocks = wal_segment_size / XLOG_BLCKSZ;
-       for (int i = 0; i < blocks;)
+       else
+#if 0
        {
-           int         iovcnt = Min(blocks - i, lengthof(iov));
-           off_t       offset = i * XLOG_BLCKSZ;
+           struct iovec iov[PG_IOV_MAX];
+           int         blocks;
 
-           if (pg_pwritev_with_retry(fd, iov, iovcnt, offset) < 0)
+           /* Prepare to write out a lot of copies of our zero buffer at once. */
+           for (int i = 0; i < lengthof(iov); ++i)
            {
-               save_errno = errno;
-               break;
+               iov[i].iov_base = XLogCtl->zerobuf;
+               iov[i].iov_len = XLOG_BLCKSZ;
+           }
+
+           /* Loop, writing as many blocks as we can for each system call. */
+           blocks = wal_segment_size / XLOG_BLCKSZ;
+           for (int i = 0; i < blocks;)
+           {
+               int         iovcnt = Min(blocks - i, lengthof(iov));
+               off_t       offset = i * XLOG_BLCKSZ;
+
+               if (pg_pwritev_with_retry(fd, iov, iovcnt, offset) < 0)
+               {
+                   save_errno = errno;
+                   break;
+               }
+               i += iovcnt;
            }
+       }
+#else
+       {
+           uint32      nbytes;
+
+           for (nbytes = 0; nbytes < wal_segment_size; nbytes += XLOG_BLCKSZ)
+           {
+               PgAioInProgress *aio = pg_streaming_write_get_io(pgsw);
 
-           i += iovcnt;
+               pgaio_io_start_write_generic(aio, fd, nbytes, XLOG_BLCKSZ, XLogCtl->zerobuf, false);
+               pg_streaming_write_write(pgsw, aio, NULL);
+           }
        }
+#endif
    }
    else
    {
@@ -3422,7 +4480,7 @@ XLogFileInit(XLogSegNo logsegno, bool *use_existent, bool use_lock)
         * enough.
         */
        errno = 0;
-       if (pg_pwrite(fd, zbuffer.data, 1, wal_segment_size - 1) != 1)
+       if (pg_pwrite(fd, XLogCtl->zerobuf, 1, wal_segment_size - 1) != 1)
        {
            /* if write didn't set errno, assume no disk space */
            save_errno = errno ? errno : ENOSPC;
@@ -3447,6 +4505,22 @@ XLogFileInit(XLogSegNo logsegno, bool *use_existent, bool use_lock)
    }
 
    pgstat_report_wait_start(WAIT_EVENT_WAL_INIT_SYNC);
+#if 1
+   {
+       if (enableFsync)
+       {
+           PgAioInProgress *aio;
+
+           pg_streaming_write_wait_all(pgsw);
+
+           aio = pg_streaming_write_get_io(pgsw);
+           pgaio_io_start_fsync(aio, fd, /* barrier = */ false);
+           pg_streaming_write_write(pgsw, aio, NULL);
+       }
+       pg_streaming_write_wait_all(pgsw);
+       pg_streaming_write_free(pgsw);
+   }
+#else
    if (pg_fsync(fd) != 0)
    {
        int         save_errno = errno;
@@ -3457,6 +4531,7 @@ XLogFileInit(XLogSegNo logsegno, bool *use_existent, bool use_lock)
                (errcode_for_file_access(),
                 errmsg("could not fsync file \"%s\": %m", tmppath)));
    }
+#endif
    pgstat_report_wait_end();
 
    if (close(fd) != 0)
@@ -5128,6 +6203,7 @@ LocalProcessControlFile(bool reset)
    ReadControlFile();
 }
 
+
 /*
  * Initialization of shared memory for XLOG
  */
@@ -5157,11 +6233,16 @@ XLOGShmemSize(void)
    /* WAL insertion locks, plus alignment */
    size = add_size(size, mul_size(sizeof(WALInsertLockPadded), NUM_XLOGINSERT_LOCKS + 1));
    /* xlblocks array */
-   size = add_size(size, mul_size(sizeof(XLogRecPtr), XLOGbuffers));
+   size = add_size(size, mul_size(sizeof(XLBlockPageStatus), XLOGbuffers));
    /* extra alignment padding for XLOG I/O buffers */
    size = add_size(size, XLOG_BLCKSZ);
    /* and the buffers themselves */
    size = add_size(size, mul_size(XLOG_BLCKSZ, XLOGbuffers));
+   /* and zerobuf */
+   size = add_size(size, XLOG_BLCKSZ);
+
+   size = add_size(size, XLogIOQueueMem());
+   size = add_size(size, XLogIOQueueMem());
 
    /*
     * Note: we don't count ControlFileData, it comes out of the "slop factor"
@@ -5235,10 +6316,9 @@ XLOGShmemInit(void)
     * needed here.
     */
    allocptr = ((char *) XLogCtl) + sizeof(XLogCtlData);
-   XLogCtl->xlblocks = (XLogRecPtr *) allocptr;
-   memset(XLogCtl->xlblocks, 0, sizeof(XLogRecPtr) * XLOGbuffers);
-   allocptr += sizeof(XLogRecPtr) * XLOGbuffers;
-
+   XLogCtl->xlblocks = (XLBlockPageStatus *) allocptr;
+   memset(XLogCtl->xlblocks, 0, sizeof(XLBlockPageStatus) * XLOGbuffers);
+   allocptr += sizeof(XLBlockPageStatus) * XLOGbuffers;
 
    /* WAL insertion locks. Ensure they're aligned to the full padded size */
    allocptr += sizeof(WALInsertLockPadded) -
@@ -5262,6 +6342,11 @@ XLOGShmemInit(void)
    allocptr = (char *) TYPEALIGN(XLOG_BLCKSZ, allocptr);
    XLogCtl->pages = allocptr;
    memset(XLogCtl->pages, 0, (Size) XLOG_BLCKSZ * XLOGbuffers);
+   allocptr += XLOG_BLCKSZ * XLOGbuffers;
+
+   XLogCtl->zerobuf = allocptr;
+   memset(XLogCtl->zerobuf, 0, XLOG_BLCKSZ);
+   allocptr += XLOG_BLCKSZ;
 
    /*
     * Do basic initialization of XLogCtl shared data. (StartupXLOG will fill
@@ -5279,6 +6364,28 @@ XLOGShmemInit(void)
    InitSharedLatch(&XLogCtl->recoveryWakeupLatch);
 
    pg_atomic_init_u64(&XLogCtl->Insert.knownCompletedUpto, 0);
+
+   {
+       size_t capacity = pg_nextpower2_32(io_wal_concurrency);
+       size_t sz = sizeof(XLogIOQueue) +
+           sizeof(XLogIO) * capacity;
+
+       XLogCtl->writes = ShmemAlloc(sz);
+       memset(XLogCtl->writes, 0, sz);
+       XLogCtl->writes->mask = capacity - 1;
+       XLogCtl->writes->num_ios = io_wal_concurrency;
+       for (int i = 0; i < capacity; i++)
+           pgaio_io_ref_clear(&XLogCtl->writes->ios[i].aio_ref);
+       // FIXME: check whether lock is needed and initialize if so
+
+       XLogCtl->flushes = ShmemAlloc(sz);
+       memset(XLogCtl->flushes, 0, sz);
+       XLogCtl->flushes->mask = capacity - 1;
+       XLogCtl->flushes->num_ios = io_wal_concurrency;
+       for (int i = 0; i < io_wal_concurrency; i++)
+           pgaio_io_ref_clear(&XLogCtl->flushes->ios[i].aio_ref);
+       // DITO
+   }
 }
 
 /*
@@ -7778,7 +8885,7 @@ StartupXLOG(void)
        memcpy(page, xlogreader->readBuf, len);
        memset(page + len, 0, XLOG_BLCKSZ - len);
 
-       XLogCtl->xlblocks[firstIdx] = pageBeginPtr + XLOG_BLCKSZ;
+       XLogCtl->xlblocks[firstIdx].lsn = pageBeginPtr + XLOG_BLCKSZ;
        XLogCtl->InitializedUpTo = pageBeginPtr + XLOG_BLCKSZ;
    }
    else
@@ -7791,12 +8898,15 @@ StartupXLOG(void)
        XLogCtl->InitializedUpTo = EndOfLog;
    }
 
-   LogwrtResult.Write = LogwrtResult.Flush = EndOfLog;
+   LogwrtResult.WriteDone = LogwrtResult.WriteInit = EndOfLog;
+   LogwrtResult.FlushDone = LogwrtResult.FlushInit = EndOfLog;
 
    XLogCtl->LogwrtResult = LogwrtResult;
 
-   XLogCtl->LogwrtRqst.Write = EndOfLog;
-   XLogCtl->LogwrtRqst.Flush = EndOfLog;
+   XLogCtl->LogwrtRqst.WriteInit = EndOfLog;
+   XLogCtl->LogwrtRqst.WriteDone = EndOfLog;
+   XLogCtl->LogwrtRqst.FlushInit = EndOfLog;
+   XLogCtl->LogwrtRqst.FlushDone = EndOfLog;
 
    /*
     * Update full_page_writes in shared memory and write an XLOG_FPW_CHANGE
@@ -8502,7 +9612,7 @@ GetInsertRecPtr(void)
    XLogRecPtr  recptr;
 
    SpinLockAcquire(&XLogCtl->info_lck);
-   recptr = XLogCtl->LogwrtRqst.Write;
+   recptr = XLogCtl->LogwrtRqst.WriteDone;
    SpinLockRelease(&XLogCtl->info_lck);
 
    return recptr;
@@ -8519,7 +9629,7 @@ GetFlushRecPtr(void)
    LogwrtResult = XLogCtl->LogwrtResult;
    SpinLockRelease(&XLogCtl->info_lck);
 
-   return LogwrtResult.Flush;
+   return LogwrtResult.FlushDone;
 }
 
 /*
@@ -10583,7 +11693,6 @@ assign_xlog_sync_method(int new_sync_method, void *extra)
    }
 }
 
-
 /*
  * Issue appropriate kind of fsync (if any) for an XLOG output file.
  *
@@ -10595,7 +11704,13 @@ issue_xlog_fsync(int fd, XLogSegNo segno)
 {
    char       *msg = NULL;
 
+   if (!enableFsync ||
+       sync_method == SYNC_METHOD_OPEN ||
+       sync_method == SYNC_METHOD_OPEN_DSYNC)
+       return;
+
    pgstat_report_wait_start(WAIT_EVENT_WAL_SYNC);
+
    switch (sync_method)
    {
        case SYNC_METHOD_FSYNC:
@@ -11704,7 +12819,7 @@ GetXLogWriteRecPtr(void)
    LogwrtResult = XLogCtl->LogwrtResult;
    SpinLockRelease(&XLogCtl->info_lck);
 
-   return LogwrtResult.Write;
+   return LogwrtResult.WriteDone;
 }
 
 /*
index 1b477d26cfe8e9eeb88ee0361821104fc5973f63..679d7e4b202e3026814da8b84a21be2a3aaed3f0 100644 (file)
@@ -163,6 +163,8 @@ bool        io_data_direct = 0;
 bool       io_data_force_async = 1;
 bool       io_wal_direct = 0;
 bool       io_wal_init_direct = 0;
+int            io_wal_concurrency = 32;
+int            io_wal_target_blocks = 8;
 
 /* local state for StartBufferIO and related functions */
 static BufferDesc *InProgressBuf = NULL;
index b7374aca857632c895556e37062e7901d360f641..bdded4399c252c8e5d25bdd36e8a31eeb16d5d36 100644 (file)
@@ -1399,6 +1399,8 @@ pgaio_can_be_combined(PgAioInProgress *last, PgAioInProgress *cur)
            return true;
 
        case PGAIO_WRITE_WAL:
+           /* FIXME: XLOG sometimes intentionally does smaller writes - this would undo that */
+           return false;
            if (last->d.write_wal.fd != cur->d.write_wal.fd)
                return false;
            if ((last->d.write_wal.offset + last->d.write_wal.nbytes) != cur->d.write_wal.offset)
index 9e53e0a8fbae63e395d21f455a2389eff1fb0dd6..9241033b563d19337c5a3cf84762da5f4b7e542e 100644 (file)
@@ -54,4 +54,4 @@ XactTruncationLock                    44
 WrapLimitsVacuumLock               46
 NotifyQueueTailLock                    47
 SharedAIOCtlLock                   48
-WALSyncLock                        49
+WALIOQueueLock                     49
index d0b6b2c9066a9a0283f9f4dc5b00b5f62be8f3c4..7f8369e939499c17cb780a92b6b7f5158475d7d1 100644 (file)
@@ -3029,6 +3029,28 @@ static struct config_int ConfigureNamesInt[] =
        check_maintenance_io_concurrency, NULL, NULL
    },
 
+   {
+       {"io_wal_concurrency", PGC_POSTMASTER, RESOURCES_ASYNCHRONOUS,
+           gettext_noop("How many concurrent IOs for WAL writes."),
+           NULL,
+           0
+       },
+       &io_wal_concurrency,
+       32, 0, INT_MAX,
+       NULL, NULL, NULL
+   },
+
+   {
+       {"io_wal_target_blocks", PGC_POSTMASTER, RESOURCES_ASYNCHRONOUS,
+           gettext_noop("Maximum size for WAL IO."),
+           NULL,
+           GUC_UNIT_XBLOCKS
+       },
+       &io_wal_target_blocks,
+       8, 0, INT_MAX,
+       NULL, NULL, NULL
+   },
+
    {
        {"backend_flush_after", PGC_USERSET, RESOURCES_ASYNCHRONOUS,
            gettext_noop("Number of pages after which previously performed writes are flushed to disk."),
index 2fd1b0c076b99e367b5125e6db42a6dbd73d73e9..bd868ea2735f81fbd09a16b9d4abe93ce789d5af 100644 (file)
@@ -80,6 +80,8 @@ extern bool io_data_direct;
 extern bool io_data_force_async;
 extern bool io_wal_direct;
 extern bool io_wal_init_direct;
+extern int io_wal_concurrency;
+extern int io_wal_target_blocks;
 
 /* in buf_init.c */
 extern PGDLLIMPORT char *BufferBlocks;
index 0786fcf103a708542f6b2dd5bf751725460c027d..cbf3ac8f3ee4516982909501d7dd28bb9ed38fc8 100644 (file)
@@ -173,6 +173,14 @@ struct PGPROC
    /* Support for condition variables. */
    proclist_node cvWaitLink;   /* position in CV wait list */
 
+   /*
+    * Support for waiting until a specific WAL location has been written /
+    * flushed (or WALwriteLock is released).
+   */
+   XLogRecPtr xlog_write_wait;
+   XLogRecPtr xlog_flush_wait;
+   proclist_node xlogFlushLink;    /* position in xlog flush wait list */
+
    /* Info about lock the process is currently waiting for, if any. */
    /* waitLock and waitProcLock are NULL if not currently waiting. */
    LOCK       *waitLock;       /* Lock object we're sleeping on ... */